-
Notifications
You must be signed in to change notification settings - Fork 13.5k
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-37403] Convert interval join to regular join for updating source #26230
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -247,9 +247,9 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti | |
val children = visitChildren(cep, ModifyKindSetTrait.INSERT_ONLY, "Match Recognize") | ||
createNewNode(cep, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) | ||
|
||
case _: StreamPhysicalTemporalSort | _: StreamPhysicalIntervalJoin | | ||
_: StreamPhysicalOverAggregate | _: StreamPhysicalPythonOverAggregate => | ||
// TemporalSort, OverAggregate, IntervalJoin only support consuming insert-only | ||
case _: StreamPhysicalTemporalSort | _: StreamPhysicalOverAggregate | | ||
_: StreamPhysicalPythonOverAggregate => | ||
// TemporalSort, and OverAggregate only support consuming insert-only | ||
// and producing insert-only changes | ||
val children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY) | ||
createNewNode(rel, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) | ||
|
@@ -287,6 +287,39 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti | |
val leftTrait = children.head.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) | ||
createNewNode(temporalJoin, children, leftTrait, requiredTrait, requester) | ||
|
||
case intervalJoin: StreamPhysicalIntervalJoin => | ||
// IntervalJoin only support consuming insert-only and producing insert-only changes | ||
// However, when the input contains updates convert to RegularJoin | ||
val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) | ||
val inputModifyKindSet = ModifyKindSet.union(children.map(getModifyKindSet): _*) | ||
val containsUpdatesOrDeletes = inputModifyKindSet.contains( | ||
ModifyKind.UPDATE) || inputModifyKindSet.contains(ModifyKind.DELETE) | ||
if (containsUpdatesOrDeletes) { | ||
// Convert to regular join if the input contains Updates | ||
val isInnerJoin = intervalJoin.joinSpec.getJoinType == FlinkJoinType.INNER | ||
val providedTrait = if (isInnerJoin) { | ||
// forward left and right modify operations | ||
val leftKindSet = getModifyKindSet(children.head) | ||
val rightKindSet = getModifyKindSet(children.last) | ||
new ModifyKindSetTrait(leftKindSet.union(rightKindSet)) | ||
} else { | ||
// otherwise, it may produce any kinds of changes | ||
ModifyKindSetTrait.ALL_CHANGES | ||
} | ||
val regularJoin = new StreamPhysicalJoin( | ||
rel.getCluster, | ||
rel.getTraitSet, | ||
intervalJoin.getLeft, | ||
intervalJoin.getRight, | ||
intervalJoin.getCondition, | ||
intervalJoin.getJoinType, | ||
intervalJoin.getHints) | ||
createNewNode(regularJoin, children, providedTrait, requiredTrait, requester) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It might be that this approach has one major issue. What is the data type of the timestamp column? Is it still marked as a time attribute or is it a regular column? If it is still marked as a time attribute, windows or subsequent interval joins would still be allowed by the planner. Of course the output would be garbage in this case. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added a commit to materialize time attributes while processing IntervalJoin if the source has updates. |
||
} else { | ||
// IntervalJoin can only support Inserts | ||
createNewNode(rel, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) | ||
} | ||
|
||
case _: StreamPhysicalCalcBase | _: StreamPhysicalCorrelateBase | | ||
_: StreamPhysicalLookupJoin | _: StreamPhysicalExchange | _: StreamPhysicalExpand | | ||
_: StreamPhysicalMiniBatchAssigner | _: StreamPhysicalWatermarkAssigner | | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,7 +35,13 @@ public IntervalJoinRestoreTest() { | |
public List<TableTestProgram> programs() { | ||
return Arrays.asList( | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_EVENT_TIME, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_EVENT_TIME_UPDATING_SOURCE, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't need a restore test. Restoring doesn't need to be tested. |
||
IntervalJoinTestPrograms | ||
.INTERVAL_JOIN_EVENT_TIME_UPDATING_SOURCE_AND_SINK_MATERIALIZE, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_PROC_TIME, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_NEGATIVE_INTERVAL); | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_PROC_TIME_UPDATING_SOURCE, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_NEGATIVE_INTERVAL, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_NEGATIVE_INTERVAL_UPDATING_SOURCE, | ||
IntervalJoinTestPrograms.INTERVAL_JOIN_NON_TIME_ATTRIBUTE); | ||
} | ||
} |
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,396 @@ | ||
{ | ||
"flinkVersion" : "2.1", | ||
"nodes" : [ { | ||
"id" : 45, | ||
"type" : "stream-exec-table-source-scan_1", | ||
"scanTableSource" : { | ||
"table" : { | ||
"identifier" : "`default_catalog`.`default_database`.`orders_t`", | ||
"resolvedTable" : { | ||
"schema" : { | ||
"columns" : [ { | ||
"name" : "id", | ||
"dataType" : "INT" | ||
}, { | ||
"name" : "order_ts_str", | ||
"dataType" : "VARCHAR(2147483647)" | ||
}, { | ||
"name" : "order_ts", | ||
"kind" : "COMPUTED", | ||
"expression" : { | ||
"rexNode" : { | ||
"kind" : "CALL", | ||
"internalName" : "$TO_TIMESTAMP$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
}, | ||
"serializableString" : "TO_TIMESTAMP(`order_ts_str`)" | ||
} | ||
} ], | ||
"watermarkSpecs" : [ ] | ||
}, | ||
"partitionKeys" : [ ] | ||
} | ||
} | ||
}, | ||
"outputType" : "ROW<`id` INT, `order_ts_str` VARCHAR(2147483647)>", | ||
"description" : "TableSourceScan(table=[[default_catalog, default_database, orders_t]], fields=[id, order_ts_str])", | ||
"inputProperties" : [ ] | ||
}, { | ||
"id" : 46, | ||
"type" : "stream-exec-calc_1", | ||
"projection" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 0, | ||
"type" : "INT" | ||
}, { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
}, { | ||
"kind" : "CALL", | ||
"internalName" : "$TO_TIMESTAMP$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
} ], | ||
"condition" : null, | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`id` INT, `order_ts_str` VARCHAR(2147483647), `order_ts` TIMESTAMP(3)>", | ||
"description" : "Calc(select=[id, order_ts_str, TO_TIMESTAMP(order_ts_str) AS order_ts])" | ||
}, { | ||
"id" : 47, | ||
"type" : "stream-exec-exchange_1", | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "HASH", | ||
"keys" : [ 0 ] | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`id` INT, `order_ts_str` VARCHAR(2147483647), `order_ts` TIMESTAMP(3)>", | ||
"description" : "Exchange(distribution=[hash[id]])" | ||
}, { | ||
"id" : 48, | ||
"type" : "stream-exec-table-source-scan_1", | ||
"scanTableSource" : { | ||
"table" : { | ||
"identifier" : "`default_catalog`.`default_database`.`shipments_t`", | ||
"resolvedTable" : { | ||
"schema" : { | ||
"columns" : [ { | ||
"name" : "id", | ||
"dataType" : "INT" | ||
}, { | ||
"name" : "order_id", | ||
"dataType" : "INT" | ||
}, { | ||
"name" : "shipment_ts_str", | ||
"dataType" : "VARCHAR(2147483647)" | ||
}, { | ||
"name" : "shipment_ts", | ||
"kind" : "COMPUTED", | ||
"expression" : { | ||
"rexNode" : { | ||
"kind" : "CALL", | ||
"internalName" : "$TO_TIMESTAMP$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 2, | ||
"type" : "VARCHAR(2147483647)" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
}, | ||
"serializableString" : "TO_TIMESTAMP(`shipment_ts_str`)" | ||
} | ||
} ], | ||
"watermarkSpecs" : [ ] | ||
}, | ||
"partitionKeys" : [ ] | ||
} | ||
}, | ||
"abilities" : [ { | ||
"type" : "ProjectPushDown", | ||
"projectedFields" : [ [ 1 ], [ 2 ] ], | ||
"producedType" : "ROW<`order_id` INT, `shipment_ts_str` VARCHAR(2147483647)> NOT NULL" | ||
}, { | ||
"type" : "ReadingMetadata", | ||
"metadataKeys" : [ ], | ||
"producedType" : "ROW<`order_id` INT, `shipment_ts_str` VARCHAR(2147483647)> NOT NULL" | ||
} ] | ||
}, | ||
"outputType" : "ROW<`order_id` INT, `shipment_ts_str` VARCHAR(2147483647)>", | ||
"description" : "TableSourceScan(table=[[default_catalog, default_database, shipments_t, project=[order_id, shipment_ts_str], metadata=[]]], fields=[order_id, shipment_ts_str])", | ||
"inputProperties" : [ ] | ||
}, { | ||
"id" : 49, | ||
"type" : "stream-exec-calc_1", | ||
"projection" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 0, | ||
"type" : "INT" | ||
}, { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
}, { | ||
"kind" : "CALL", | ||
"internalName" : "$TO_TIMESTAMP$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
} ], | ||
"condition" : null, | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`order_id` INT, `shipment_ts_str` VARCHAR(2147483647), `shipment_ts` TIMESTAMP(3)>", | ||
"description" : "Calc(select=[order_id, shipment_ts_str, TO_TIMESTAMP(shipment_ts_str) AS shipment_ts])" | ||
}, { | ||
"id" : 50, | ||
"type" : "stream-exec-exchange_1", | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "HASH", | ||
"keys" : [ 0 ] | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`order_id` INT, `shipment_ts_str` VARCHAR(2147483647), `shipment_ts` TIMESTAMP(3)>", | ||
"description" : "Exchange(distribution=[hash[order_id]])" | ||
}, { | ||
"id" : 51, | ||
"type" : "stream-exec-join_1", | ||
"joinSpec" : { | ||
"joinType" : "INNER", | ||
"leftKeys" : [ 0 ], | ||
"rightKeys" : [ 0 ], | ||
"filterNulls" : [ true ], | ||
"nonEquiCondition" : { | ||
"kind" : "CALL", | ||
"syntax" : "BINARY", | ||
"internalName" : "$AND$1", | ||
"operands" : [ { | ||
"kind" : "CALL", | ||
"syntax" : "BINARY", | ||
"internalName" : "$>=$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 2, | ||
"type" : "TIMESTAMP(3)" | ||
}, { | ||
"kind" : "CALL", | ||
"syntax" : "SPECIAL", | ||
"internalName" : "$-$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 5, | ||
"type" : "TIMESTAMP(3)" | ||
}, { | ||
"kind" : "LITERAL", | ||
"value" : "5000", | ||
"type" : "INTERVAL SECOND(6) NOT NULL" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
} ], | ||
"type" : "BOOLEAN" | ||
}, { | ||
"kind" : "CALL", | ||
"syntax" : "BINARY", | ||
"internalName" : "$<=$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 2, | ||
"type" : "TIMESTAMP(3)" | ||
}, { | ||
"kind" : "CALL", | ||
"syntax" : "SPECIAL", | ||
"internalName" : "$+$1", | ||
"operands" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 5, | ||
"type" : "TIMESTAMP(3)" | ||
}, { | ||
"kind" : "LITERAL", | ||
"value" : "5000", | ||
"type" : "INTERVAL SECOND(6) NOT NULL" | ||
} ], | ||
"type" : "TIMESTAMP(3)" | ||
} ], | ||
"type" : "BOOLEAN" | ||
} ], | ||
"type" : "BOOLEAN" | ||
} | ||
}, | ||
"state" : [ { | ||
"index" : 0, | ||
"ttl" : "0 ms", | ||
"name" : "leftState" | ||
}, { | ||
"index" : 1, | ||
"ttl" : "0 ms", | ||
"name" : "rightState" | ||
} ], | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
}, { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`id` INT, `order_ts_str` VARCHAR(2147483647), `order_ts` TIMESTAMP(3), `order_id` INT, `shipment_ts_str` VARCHAR(2147483647), `shipment_ts` TIMESTAMP(3)>", | ||
"description" : "Join(joinType=[InnerJoin], where=[((id = order_id) AND (order_ts >= (shipment_ts - 5000:INTERVAL SECOND)) AND (order_ts <= (shipment_ts + 5000:INTERVAL SECOND)))], select=[id, order_ts_str, order_ts, order_id, shipment_ts_str, shipment_ts], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" | ||
}, { | ||
"id" : 52, | ||
"type" : "stream-exec-calc_1", | ||
"projection" : [ { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 0, | ||
"type" : "INT" | ||
}, { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 1, | ||
"type" : "VARCHAR(2147483647)" | ||
}, { | ||
"kind" : "INPUT_REF", | ||
"inputIndex" : 4, | ||
"type" : "VARCHAR(2147483647)" | ||
} ], | ||
"condition" : null, | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`order_id` INT, `order_ts_str` VARCHAR(2147483647), `shipment_ts_str` VARCHAR(2147483647)>", | ||
"description" : "Calc(select=[id AS order_id, order_ts_str, shipment_ts_str])" | ||
}, { | ||
"id" : 53, | ||
"type" : "stream-exec-sink_1", | ||
"configuration" : { | ||
"table.exec.sink.keyed-shuffle" : "AUTO", | ||
"table.exec.sink.not-null-enforcer" : "ERROR", | ||
"table.exec.sink.rowtime-inserter" : "ENABLED", | ||
"table.exec.sink.type-length-enforcer" : "IGNORE", | ||
"table.exec.sink.upsert-materialize" : "AUTO" | ||
}, | ||
"dynamicTableSink" : { | ||
"table" : { | ||
"identifier" : "`default_catalog`.`default_database`.`sink_t`", | ||
"resolvedTable" : { | ||
"schema" : { | ||
"columns" : [ { | ||
"name" : "order_id", | ||
"dataType" : "INT" | ||
}, { | ||
"name" : "order_ts_str", | ||
"dataType" : "VARCHAR(2147483647)" | ||
}, { | ||
"name" : "shipment_ts_str", | ||
"dataType" : "VARCHAR(2147483647)" | ||
} ], | ||
"watermarkSpecs" : [ ] | ||
}, | ||
"partitionKeys" : [ ] | ||
} | ||
} | ||
}, | ||
"inputChangelogMode" : [ "INSERT" ], | ||
"inputProperties" : [ { | ||
"requiredDistribution" : { | ||
"type" : "UNKNOWN" | ||
}, | ||
"damBehavior" : "PIPELINED", | ||
"priority" : 0 | ||
} ], | ||
"outputType" : "ROW<`order_id` INT, `order_ts_str` VARCHAR(2147483647), `shipment_ts_str` VARCHAR(2147483647)>", | ||
"description" : "Sink(table=[default_catalog.default_database.sink_t], fields=[order_id, order_ts_str, shipment_ts_str])" | ||
} ], | ||
"edges" : [ { | ||
"source" : 45, | ||
"target" : 46, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 46, | ||
"target" : 47, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 48, | ||
"target" : 49, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 49, | ||
"target" : 50, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 47, | ||
"target" : 51, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 50, | ||
"target" : 51, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 51, | ||
"target" : 52, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
}, { | ||
"source" : 52, | ||
"target" : 53, | ||
"shuffle" : { | ||
"type" : "FORWARD" | ||
}, | ||
"shuffleMode" : "PIPELINED" | ||
} ] | ||
} |
Large diffs are not rendered by default.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why only inner and not all?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is the same as the RegularJoin condition
RegularJoin has conditions for INNER and SEMI. However, IntervalJoin doesn't support SEMI
Ref: https://github.com/apache/flink/pull/26230/files#diff-ddf95eb3949ab889e8e3bccdacb9f57553aac06657574fd6316ca17dd48321a1R262