[
https://issues.apache.org/jira/browse/FLINK-7797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16305855#comment-16305855
]
ASF GitHub Bot commented on FLINK-7797:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/5140#discussion_r159012517
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala
---
@@ -241,17 +288,66 @@ abstract class TimeBoundedStreamInnerJoin(
if (leftTime >= leftQualifiedLowerBound && leftTime <=
leftQualifiedUpperBound) {
val leftRows = leftEntry.getValue
var i = 0
+ var markEmitted = false
while (i < leftRows.size) {
- joinFunction.join(leftRows.get(i), rightRow, cRowWrapper)
+ joinCollector.resetThisTurn()
+ val tuple = leftRows.get(i)
+ joinFunction.join(tuple.f0, rightRow, joinCollector)
+ if (joinType == JoinType.LEFT_OUTER || joinType ==
JoinType.FULL_OUTER) {
+ if (!tuple.f1 && joinCollector.everEmittedThisTurn) {
+ // Mark the left row as being successfully joined and
emitted.
+ tuple.f1 = true
+ markEmitted = true
+ }
+ }
i += 1
}
+ if (markEmitted) {
+ // Write back the edited entry (mark emitted) for the right
cache.
+ leftEntry.setValue(leftRows)
+ }
}
if (leftTime <= leftExpirationTime) {
+ if (joinType == JoinType.RIGHT_OUTER || joinType ==
JoinType.FULL_OUTER) {
--- End diff --
`JoinType.RIGHT_OUTER` should be `JoinType.LEFT_OUTER`
> Add support for windowed outer joins for streaming tables
> ---------------------------------------------------------
>
> Key: FLINK-7797
> URL: https://issues.apache.org/jira/browse/FLINK-7797
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Affects Versions: 1.4.0
> Reporter: Fabian Hueske
> Assignee: Xingcan Cui
>
> Currently, only windowed inner joins for streaming tables are supported.
> This issue is about adding support for windowed LEFT, RIGHT, and FULL OUTER
> joins.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)