You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Aljoscha Krettek (JIRA)" <ji...@apache.org> on 2017/07/11 09:21:00 UTC

[jira] [Commented] (FLINK-5031) Consecutive DataStream.split() ignored

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

Aljoscha Krettek commented on FLINK-5031:
-----------------------------------------

[~zivmeri] There was no decision yet, I only highlighted what the current version actually does (which is highly unexpected) and also highlighted why changing this to be the "union" of all splits and selects is a bit hard.

I proposed to remove this feature because Flink has side outputs since version 1.3: https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/stream/side_output.html which are a strictly more powerful API.

What do you think about this?

> Consecutive DataStream.split() ignored
> --------------------------------------
>
>                 Key: FLINK-5031
>                 URL: https://issues.apache.org/jira/browse/FLINK-5031
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API
>    Affects Versions: 1.2.0, 1.1.3
>            Reporter: Fabian Hueske
>            Assignee: Renkai Ge
>
> The output of the following program 
> {code}
> static final class ThresholdSelector implements OutputSelector<Long> {
> 	long threshold;
> 	public ThresholdSelector(long threshold) {
> 		this.threshold = threshold;
> 	}
> 	@Override
> 	public Iterable<String> select(Long value) {
> 		if (value < threshold) {
> 			return Collections.singletonList("Less");
> 		} else {
> 			return Collections.singletonList("GreaterEqual");
> 		}
> 	}
> }
> public static void main(String[] args) throws Exception {
> 	StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
> 	env.setParallelism(1);
> 	SplitStream<Long> split1 = env.generateSequence(1, 11)
> 		.split(new ThresholdSelector(6));
> 	// stream11 should be [1,2,3,4,5]
> 	DataStream<Long> stream11 = split1.select("Less");
> 	SplitStream<Long> split2 = stream11
> //		.map(new MapFunction<Long, Long>() {
> //			@Override
> //			public Long map(Long value) throws Exception {
> //				return value;
> //			}
> //		})
> 		.split(new ThresholdSelector(3));
> 	DataStream<Long> stream21 = split2.select("Less");
> 	// stream21 should be [1,2]
> 	stream21.print();
> 	env.execute();
> }
> {code}
> should be {{1, 2}}, however it is {{1, 2, 3, 4, 5}}. It seems that the second {{split}} operation is ignored.
> The program is correctly evaluate if the identity {{MapFunction}} is added to the program.



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