[
https://issues.apache.org/jira/browse/FLINK-3315?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15159154#comment-15159154
]
ASF GitHub Bot commented on FLINK-3315:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/1641#discussion_r53808877
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
---
@@ -533,4 +563,33 @@ private StreamGraph
generateInternal(List<StreamTransformation<?>> transformatio
return Collections.singleton(transform.getId());
}
+ /**
+ * Determines the slot sharing group for an operation based on the slot
sharing group set by
+ * the user and the slot sharing groups of the inputs.
+ *
+ * <p>If the user specifies a group name, this is taken as is. If
nothing is specified and
+ * the input operations all have the same group name then this name is
taken. Otherwise the
+ * default group is choosen.
+ *
+ * @param specifiedGroup The group specified by the user.
+ * @param inputIds The IDs of the input operations.
+ */
+ private String determineSlotSharingGroup(String specifiedGroup,
Collection<Integer> inputIds) {
+ if (specifiedGroup != null) {
+ return specifiedGroup;
+ } else {
+ String inputGroup = null;
+ for (int id: inputIds) {
+ String inputGroupCandidate =
streamGraph.getSlotSharingGroup(id);
+ if (inputGroup == null) {
+ inputGroup = inputGroupCandidate;
+ continue;
+ }
+ if (!inputGroup.equals(inputGroupCandidate)) {
--- End diff --
Can't we get rid of the `continue` statement by making this an `else
if(...)` branch?
> Fix Slot Sharing in Streaming API
> ---------------------------------
>
> Key: FLINK-3315
> URL: https://issues.apache.org/jira/browse/FLINK-3315
> Project: Flink
> Issue Type: Improvement
> Components: Streaming
> Affects Versions: 1.0.0
> Reporter: Aljoscha Krettek
> Assignee: Aljoscha Krettek
> Priority: Blocker
>
> Right now, the slot sharing/resource group logic is a bit "nebulous". The
> slot sharing group that operators are put in depends on the order in which
> operations are created. For example, in this case:
> {code}
> Source a = env.source()
> Source b = env.source()
> a.map().startNewResourceGroup().sink()
> b.map().sink()
> {code}
> We end up with two resource groups:
> - group 1: source a
> - group 2: map(), sink(), source b, map(), sink()
> The reason is that the slot sharing id is incremented when transforming the
> {{startNewResouceGroup()}} call and all operators that are transformed
> afterwards in graph traversal get that new slot sharing id.
> (There is also {{isolateResources()}} which can be used to isolate an
> operator.)
> What I propose is to remove {{startNewResourceGroup()}} and
> {{isolateResouces()}} and replace it with {{slotSharingGroup(String)}}. By
> default, operations would be in slot sharing group "default". This allows
> very fine grained control over what operators end up in which slot sharing
> group. For example, I could have this topology:
> {code}
> Source a = env.source().slotSharingGroup("sources")
> Source b = env.source().slotSharingGroup("sources")
> a.map().slotSharingGroup("heavy a").sink().slotSharingGroup("sinks")
> b.map().slotSharingGroup("heavy b").sink().slotSharingGroup("sinks")
> {code}
> Which would isolate the lightweight sources and sinks in a group and put
> heavy operations inside their own slot groups.
> This is a bit more low level than the previous API and requires more calls
> than a simple {{startNewResourceGroup()}} but I think not many people would
> use this feature and this design makes it very clear what operations end up
> in the same group.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)