[
https://issues.apache.org/jira/browse/FLINK-3538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15172118#comment-15172118
]
ASF GitHub Bot commented on FLINK-3538:
---------------------------------------
GitHub user aljoscha opened a pull request:
https://github.com/apache/flink/pull/1740
[FLINK-3538] Streamline Scala DataStream.join/coGroup
This enforces that the user always has to specify keys for both inputs
before .window() can be called.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/aljoscha/flink scala-stream-join-fix
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/1740.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #1740
----
commit 70391df2979ee59d6ddad96c8fcb0e6a5a74ebf1
Author: Aljoscha Krettek <[email protected]>
Date: 2016-02-29T16:02:38Z
[FLINK-3538] Streamline Scala DataStream.join/coGroup
This enforces that the user always has to specify keys for both inputs
before .window() can be called.
----
> DataStream join API does not enforce consistent usage
> -----------------------------------------------------
>
> Key: FLINK-3538
> URL: https://issues.apache.org/jira/browse/FLINK-3538
> Project: Flink
> Issue Type: Improvement
> Components: DataStream API, Scala API
> Affects Versions: 1.0.0
> Reporter: Till Rohrmann
> Assignee: Aljoscha Krettek
>
> In the Scala DataStream API the {{join}} operation does not enforce that the
> user has specified a {{KeySelector}} for both input sides before applying a
> window function. Moreover, the order of the {{where}} and {{equalTo}} clause
> is not fixed and it is possible to specify multiple {{where}} and {{equalTo}}
> clauses. In the latter case, it is not clear which {{KeySelector}} will
> eventually be used by the system.
> So the following Flink programs compile without a compilation problem (the
> first two lines will only fail at runtime):
> {code}
> inputA.join(inputB).equalTo{x =>
> x}.window(TumblingProcessingTimeWindows.of(Time.seconds(10)))
> .apply(new DefaultFlatJoinFunction[String, String]()).print()
> inputA.join(inputB).where{x =>
> x}.window(TumblingProcessingTimeWindows.of(Time.seconds(10)))
> .apply(new DefaultFlatJoinFunction[String, String]()).print()
> inputA.join(inputB).equalTo{x => x}.where{x => x}.where{x => "1"}.equalTo{x
> => "42"}.window(TumblingProcessingTimeWindows.of(Time.seconds(10)))
> .apply(new DefaultFlatJoinFunction[String, String]()).print()
> {code}
> This is unlike the Java DataStream API where a clear pattern of {{join}} then
> {{where}} and then {{equalTo}} is enforced. I would propose to do the same
> for the Scala API.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)