[
https://issues.apache.org/jira/browse/FLINK-10050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16614875#comment-16614875
]
ASF GitHub Bot commented on FLINK-10050:
----------------------------------------
kl0u commented on a change in pull request #6646: [FLINK-10050] Support
allowedLateness in CoGroupedStreams
URL: https://github.com/apache/flink/pull/6646#discussion_r217716951
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
##########
@@ -254,7 +260,17 @@ protected WithWindow(DataStream<T1> input1,
@PublicEvolving
public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super
TaggedUnion<T1, T2>, ? super W> newEvictor) {
return new WithWindow<>(input1, input2, keySelector1,
keySelector2, keyType,
- windowAssigner, trigger, newEvictor);
+ windowAssigner, trigger, newEvictor,
allowedLateness);
+ }
+
+ /**
+ * Sets the time by which elements are allowed to be late.
+ * @see WindowedStream#allowedLateness(Time)
+ */
+ @PublicEvolving
+ public WithWindow<T1, T2, KEY, W> allowedLateness(Time
newLateness) {
+ return new WithWindow<>(input1, input2, keySelector1,
keySelector2, keyType,
Review comment:
Check for `null`. If the user calls `allowedLateness`, he should not pass a
`null` value.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> Support 'allowedLateness' in CoGroupedStreams
> ---------------------------------------------
>
> Key: FLINK-10050
> URL: https://issues.apache.org/jira/browse/FLINK-10050
> Project: Flink
> Issue Type: Improvement
> Components: Streaming
> Affects Versions: 1.5.1, 1.6.0
> Reporter: eugen yushin
> Priority: Major
> Labels: pull-request-available, ready-to-commit, windows
>
> WindowedStream has a support of 'allowedLateness' feature, while
> CoGroupedStreams are not. At the mean time, WindowedStream is an inner part
> of CoGroupedStreams and all main functionality (like evictor/trigger/...) is
> simply delegated to WindowedStream.
> There's no chance to operate with late arriving data from previous steps in
> cogroups (and joins). Consider the following flow:
> a. read data from source1 -> aggregate data with allowed lateness
> b. read data from source2 -> aggregate data with allowed lateness
> c. cogroup/join streams a and b, and compare aggregated values
> Step c doesn't accept any late data from steps a/b due to lack of
> `allowedLateness` API call in CoGroupedStreams.java.
> Scope: add method `WithWindow.allowedLateness` to Java API
> (flink-streaming-java) and extend scala API (flink-streaming-scala).
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)