Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-19609][table-planner-blink] Support streaming window join in planner #15195

Merged
merged 6 commits into from
Mar 17, 2021

Conversation

beyond1920
Copy link
Contributor

What is the purpose of the change

This pull request aims to support streaming WindowJoin in planner.

Brief change log

  • Introduce StreamPhysicalWindowJoin
  • Update FlinkChangelogModeInterenceProgram and FlinkRelMdWindowProperties to take StreamPhysicalWindowJoin into consideration.
  • Introduce StreamPhysicalWindowJoinRule to convert FlinkLogicalJoin to StreamPhysicalWindowJoin if join condition contains window starts equality of input tables and window ends equality of input tables.

Verifying this change

  • UT in WindowJoinTest.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not applicable)

@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 8f844ec (Sun Mar 14 08:40:20 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Mar 14, 2021

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

Copy link
Member

@wuchong wuchong left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the great work @beyond1920 . I left some comments.

val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val leftWindowProperties = fmq.getRelWindowProperties(rel.getLeft)
val rightWindowProperties = fmq.getRelWindowProperties(rel.getRight)
assert(leftWindowProperties.getWindowSpec == rightWindowProperties.getWindowSpec)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to use assert here? I think the query is still valid, we just can't infer the window properties. I think we can return null in this case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The case will not happen because the node will not be translated to WindowJoin if WindowSpec is different.

@@ -266,6 +266,12 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
}
createNewNode(join, children, providedTrait, requiredTrait, requester)

case windowJoin: StreamPhysicalWindowJoin =>
// window join support all changes in input
val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would suggest to use INSERT_ONLY here (and we don't add any tests for this). We can

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

Comment on lines +49 to +51
val leftWindowing: WindowingStrategy,
val rightWindowing: WindowingStrategy)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that we do not allow different window strategy for left and right input. I think we can simplify the design to only have a single window strategy. We can evolve it in the future.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Left windowStrategy and Right windowStrategy only have same timeAttribute and WindowSpec. But window start index and window end index maybe different.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. You are right. Would be better to check the two window strategies in the physical node by overriding isValid.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

def containsWindowStartEqualityOrEndEquality(join: FlinkLogicalJoin): Boolean = {
val (windowStartEqualityLeftKeys, windowEndEqualityLeftKeys, _, _) =
excludeWindowStartEqualityAndEndEqualityFromJoinInfoPairs(join)
windowStartEqualityLeftKeys.nonEmpty || windowEndEqualityLeftKeys.nonEmpty
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it would be better to use &&, because we don't allow start OR end for now in excludeWindowStartEqualityAndEndEqualityFromJoinInfoPairs.

val rightFieldType = joinFieldsType.get(rightIndex).getType
val rightInputRef = new RexInputRef(rightIndex, rightFieldType)
val remainEqual = rexBuilder.makeCall(
SqlStdOperatorTable.EQUALS,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why the remainCondition should contain the join key equality?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remainCondition contains two part:

  1. remain equal condition exclude window start equality and window end equality
  2. non-equal condition

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not put the join key equality (exclude window start & end) into join key paires? I mean construct a new JoinInfo instead of a RexNode.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@wuchong Thanks, the solution you mentioned works well of course. However I prefer let StreamPhysicalWindowJoin extends CommonPhysicalJoin, so a complete condition which contains equal condition and non-equal condition is required, JoinInfo would be inferred in the Join node. What do you think?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the explanation. Sounds good to me.

<Resource name="optimized rel plan">
<![CDATA[
Calc(select=[a, b, c, a0, b0, c0])
+- WindowJoin(leftWindow=[CUMULATE(win_start=[window_start], win_end=[window_end], max_size=[1 h], step=[10 min])], rightWindow=[CUMULATE(win_start=[window_start], win_end=[window_end], max_size=[1 h], step=[10 min])], joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, c, window_start, window_end, a0, b0, c0, window_start0, window_end0])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note: when implementing window join operator, we can't support proctime window join for now, becuase the window assigner is in an separate operator, that means we don't know when to trigger the processing-time window (we don't have something like watermark for proctime). We need to merge window assigner into window join to support proctime window join, but this can be a future work.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point! Do we need to throw UnsupportedException in planner, Or just when translate to ExecNode?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

throwing a readable exception sounds good to me.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Throwing in planner or when translating to exec node are both fine. But this can be done in the next PR.

Copy link
Contributor Author

@beyond1920 beyond1920 Mar 15, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. If I understand well, the problem also exists in WindowJoin, WindowAggregate, WindowJoin when input window strategy is WindowAttachedWindowingStrategy, and TimeAttribute is proc-time at the same time. Right?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes.

<Resource name="optimized rel plan">
<![CDATA[
Calc(select=[window_start, window_end, a, cnt AS l_cnt, uv AS l_uv, cnt0 AS r_cnt, uv0 AS r_uv, rownum])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[window_start, window_end], orderBy=[cnt DESC], select=[a, window_start, window_end, window_time, cnt, uv, a0, window_start0, window_end0, window_time0, cnt0, uv0, rownum])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it mean window properties are not propagated? Because here is Rank instead of WindowRank.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch, I missed windowProperties inference for FlinkLogicalJoin in FlinkRelMdWindowProperties.

// ----------------------------------------------------------------------------------------

/** Window type in left and right child should be same **/
@Test(expected = classOf[TableException])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use thrown.expect(...) and thrown.expectMessage(..) to assert exception and error message.

@beyond1920
Copy link
Contributor Author

@wuchong Thanks for review. I've address all your comments.

@beyond1920 beyond1920 force-pushed the window-join branch 3 times, most recently from 162ca63 to 0a79f7d Compare March 15, 2021 08:35
* @return True if join condition contains window starts equality of input tables and window
* ends equality of input tables. Else false.
*/
def containsWindowStartEqualityOrEndEquality(join: FlinkLogicalJoin): Boolean = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
def containsWindowStartEqualityOrEndEquality(join: FlinkLogicalJoin): Boolean = {
def containsWindowStartEqualityAndEndEquality(join: FlinkLogicalJoin): Boolean = {

Comment on lines 195 to 198
|Currently, the windowing TVFs must be the same of left and right inputs.
|In the future, we could support different window TVFs, for example, tumbling windows
| join sliding windows with the same window size.
|""".stripMargin)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should make it clear what's the current operator, and what's the difference.

Currently, window join doesn't support different window table function of left and right inputs. The left window table function is %s, the right window table function is %s.

Comment on lines 189 to 191
|Currently, time attribute type of left and right inputs should be both row-time or
|both proc-time. In the future, we could support different time attribute type.
|""".stripMargin)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, window join doesn't support different time attribute type of left and right inputs. The left time attribute type is %s, the right time attribute type is %s.

|Currently, window starts equality and window ends equality are both required for
|window join. In the future, we could support join clause which only includes window
|starts equality or window ends equality for TUMBLE or HOP window.
|""".stripMargin)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, window join requires JOIN ON condition must contain both window starts equality of input tables and window ends equality of input tables. But the current JOIN ON condition is %s.

@beyond1920
Copy link
Contributor Author

@wuchong Thanks, I've updated based on your comment.

|starts equality or window ends equality for TUMBLE or HOP window.
|""".stripMargin)
"Currently, window join requires JOIN ON condition must contain both window starts " +
"equality of input tables and window ends equality of input tables.")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please also verify the full exception message.

|starts equality or window ends equality for TUMBLE or HOP window.
|""".stripMargin)
"Currently, window join requires JOIN ON condition must contain both window starts " +
"equality of input tables and window ends equality of input tables.")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please also verify the full exception message.

|""".stripMargin)
"Currently, window join doesn't support different window table function of left and " +
"right inputs.\n" +
s"The left window table function is ${leftWindowProperties}.\n" +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We just need to print the windowSpec instead of the total leftWindowProperties. Besides, we need to implement toString method for WindowSpecs.

|""".stripMargin)
"Currently, window join requires JOIN ON condition must contain both window starts " +
"equality of input tables and window ends equality of input tables.\n" +
s"But the current JOIN ON condition is ${join.getCondition}.")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Currently, it prints:

But the current JOIN ON condition is AND(=($2, $7), =($0, $5)).

I think it unreadable and not helpful for users. Would be better to display field names.

@beyond1920
Copy link
Contributor Author

@wuchong Thanks a lot, I've updated based on your comment.

if (windowStartEqualityLeftKeys.nonEmpty && windowEndEqualityLeftKeys.nonEmpty) {
if (
leftWindowProperties.getTimeAttributeType != rightWindowProperties.getTimeAttributeType) {

def timeAttributeTypeStr(isRowTime: Boolean): String = {
if (isRowTime) "ROWTIME" else "PROCTIME"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can just print the logical type, because we will support TIMESTAMP_LTZ as time attribute soon, so the logical type maybe different.

windowSpec: WindowSpec): String = {
val windowing = s"win_start=[${inputFieldNames(windowStartIdx)}]" +
s", win_end=[${inputFieldNames(windowEndIdx)}]"
windowSpec.toSummaryString(windowing)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

WindowSpec doesn't contain window_start and window_end column information, so I think we don't need to print them.

@@ -182,26 +183,65 @@ object WindowJoinUtil {
}

// Validate join
def getLeftFieldNames() = join.getLeft.getRowType.getFieldNames.toList

def getRightFieldNames() = join.getRight.getRowType.getFieldNames.toList
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A local variable is enough?

@wuchong
Copy link
Member

wuchong commented Mar 17, 2021

I appended a commit to improve the exception message. And also rebased the branch. If there is no objections, I will merge it once the build is passed.

Copy link
Member

@wuchong wuchong left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@beyond1920
Copy link
Contributor Author

@wuchong Thanks a lot. The update is good.

@wuchong wuchong merged commit 20cb469 into apache:master Mar 17, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants