From 815ed11de821da4512efce1d8fb4b93d650e0f09 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Tue, 24 Mar 2026 11:43:07 +0100 Subject: [PATCH 1/6] [FLINK-38209][table-planner] Use UniqueKeys instead of Upsertkeys for inputSpec and state management for MultiJoin --- .../exec/stream/StreamExecMultiJoin.java | 31 ++++---- .../stream/StreamPhysicalMultiJoin.java | 19 ++--- .../serde/ExecNodeMultiJoinJsonSerdeTest.java | 6 +- .../exec/stream/MultiJoinSemanticTests.java | 3 +- .../exec/stream/MultiJoinTestPrograms.java | 66 +++++++++++++++++- ...ay-complex-updating-join-with-restore.json | 2 +- .../savepoint/_metadata | Bin 35162 -> 35375 bytes ...-join-no-common-join-key-with-restore.json | 4 +- .../savepoint/_metadata | Bin 41248 -> 38180 bytes .../three-way-inner-join-with-restore.json | 2 +- .../savepoint/_metadata | Bin 21326 -> 21331 bytes ...oin-with-time-attributes-with-restore.json | 2 +- .../savepoint/_metadata | Bin 24946 -> 23917 bytes ...hree-way-left-outer-join-with-restore.json | 2 +- .../savepoint/_metadata | Bin 21759 -> 21588 bytes 15 files changed, 100 insertions(+), 37 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java index b6ca3571e1a7d..31e04a693c7c0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultiJoin.java @@ -82,7 +82,7 @@ public class StreamExecMultiJoin extends ExecNodeBase private static final String FIELD_NAME_JOIN_TYPES = "joinTypes"; private static final String FIELD_NAME_JOIN_CONDITIONS = "joinConditions"; private static final String FIELD_NAME_JOIN_ATTRIBUTE_MAP = "joinAttributeMap"; - private static final String FIELD_NAME_INPUT_UPSERT_KEYS = "inputUpsertKeys"; + private static final String FIELD_NAME_INPUT_UNIQUE_KEYS = "inputUniqueKeys"; private static final String FIELD_NAME_MULTI_JOIN_CONDITION = "multiJoinCondition"; @JsonProperty(FIELD_NAME_JOIN_TYPES) @@ -100,12 +100,13 @@ public class StreamExecMultiJoin extends ExecNodeBase @JsonInclude(JsonInclude.Include.NON_EMPTY) private final Map> joinAttributeMap; - @JsonProperty(FIELD_NAME_INPUT_UPSERT_KEYS) + // Why List> as a type + // Each unique key can be also a composite key with multiple fields, thus -> int[]. + // Theoretically, each input can have multiple unique keys, thus -> List + // Since we have multiple inputs -> List> + @JsonProperty(FIELD_NAME_INPUT_UNIQUE_KEYS) @JsonInclude(JsonInclude.Include.NON_EMPTY) - // List of upsert keys for each input, where each inner list corresponds to an input - // The reason it's a List> is that SQL allows only one primary key but - // multiple upsert (unique) keys per input - private final List> inputUpsertKeys; + private final List> inputUniqueKeys; @JsonProperty(FIELD_NAME_STATE) @JsonInclude(JsonInclude.Include.NON_NULL) @@ -117,7 +118,7 @@ public StreamExecMultiJoin( final List joinConditions, @Nullable final RexNode multiJoinCondition, final Map> joinAttributeMap, - final List> inputUpsertKeys, + final List> inputUniqueKeys, final Map stateTtlFromHint, final List inputProperties, final RowType outputType, @@ -130,7 +131,7 @@ public StreamExecMultiJoin( joinConditions, multiJoinCondition, joinAttributeMap, - inputUpsertKeys, + inputUniqueKeys, StateMetadata.getMultiInputOperatorDefaultMeta( stateTtlFromHint, tableConfig, generateStateNames(inputProperties.size())), inputProperties, @@ -150,18 +151,18 @@ public StreamExecMultiJoin( final RexNode multiJoinCondition, @JsonProperty(FIELD_NAME_JOIN_ATTRIBUTE_MAP) final Map> joinAttributeMap, - @JsonProperty(FIELD_NAME_INPUT_UPSERT_KEYS) final List> inputUpsertKeys, + @JsonProperty(FIELD_NAME_INPUT_UNIQUE_KEYS) final List> inputUniqueKeys, @Nullable @JsonProperty(FIELD_NAME_STATE) final List stateMetadataList, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) final List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) final RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) final String description) { super(id, context, persistedConfig, inputProperties, outputType, description); - validateInputs(inputProperties, joinTypes, joinConditions, inputUpsertKeys); + validateInputs(inputProperties, joinTypes, joinConditions, inputUniqueKeys); this.joinTypes = checkNotNull(joinTypes); this.joinConditions = checkNotNull(joinConditions); this.multiJoinCondition = multiJoinCondition; - this.inputUpsertKeys = checkNotNull(inputUpsertKeys); this.joinAttributeMap = Objects.requireNonNullElseGet(joinAttributeMap, Map::of); + this.inputUniqueKeys = checkNotNull(inputUniqueKeys); this.stateMetadataList = stateMetadataList; } @@ -169,7 +170,7 @@ private void validateInputs( final List inputProperties, final List joinTypes, final List joinConditions, - final List> inputUpsertKeys) { + final List> inputUniqueKeys) { checkArgument( inputProperties.size() >= 2, "Multi-input join operator needs at least 2 inputs."); checkArgument( @@ -179,8 +180,8 @@ private void validateInputs( joinConditions.size() == inputProperties.size(), "Size of joinConditions must match the number of inputs."); checkArgument( - inputUpsertKeys.size() == inputProperties.size(), - "Size of inputUpsertKeys must match the number of inputs."); + inputUniqueKeys.size() == inputProperties.size(), + "Size of inputUniqueKeys must match the number of inputs."); } private static String[] generateStateNames(int numInputs) { @@ -220,7 +221,7 @@ protected Transformation translateToPlanInternal( planner.getFlinkContext().getClassLoader(), inputTypeInfos.get(i), keyExtractor.getJoinKeyIndices(i), - inputUpsertKeys.get(i))); + inputUniqueKeys.get(i))); } final GeneratedJoinCondition[] generatedJoinConditions = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java index 0d5ee9c8b7615..fb25cab268f3b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java @@ -167,7 +167,7 @@ protected RelDataType deriveRowType() { @Override public ExecNode translateToExecNode() { final RexNode multiJoinCondition = createMultiJoinCondition(); - final List> inputUpsertKeys = getUpsertKeysForInputs(); + final List> inputUniqueKeys = getUniqueKeysForInputs(); final List execJoinTypes = getExecJoinTypes(); final List inputProperties = createInputProperties(); @@ -177,7 +177,7 @@ public ExecNode translateToExecNode() { joinConditions, multiJoinCondition, joinAttributeMap, - inputUpsertKeys, + inputUniqueKeys, Collections.emptyMap(), // TODO Enable hint-based state ttl. See ticket // TODO https://issues.apache.org/jira/browse/FLINK-37936 inputProperties, @@ -194,26 +194,27 @@ private RexNode createMultiJoinCondition() { return RexUtil.composeConjunction(getCluster().getRexBuilder(), conjunctions, true); } - private List> getUpsertKeysForInputs() { + private List> getUniqueKeysForInputs() { return inputs.stream() .map( input -> { - final Set upsertKeys = getUpsertKeys(input); + final Set uniqueKeys = getUniqueKeys(input); - if (upsertKeys == null) { + if (uniqueKeys == null) { return Collections.emptyList(); } - return upsertKeys.stream() + + return uniqueKeys.stream() .map(ImmutableBitSet::toArray) .collect(Collectors.toList()); }) .collect(Collectors.toList()); } - private @Nullable Set getUpsertKeys(RelNode input) { + private @Nullable Set getUniqueKeys(RelNode input) { final FlinkRelMetadataQuery fmq = FlinkRelMetadataQuery.reuseOrCreate(input.getCluster().getMetadataQuery()); - return fmq.getUpsertKeys(input); + return fmq.getUniqueKeys(input); } private List getExecJoinTypes() { @@ -255,7 +256,7 @@ public List getJoinTypes() { */ public boolean inputUniqueKeyContainsCommonJoinKey(int inputId) { final RelNode input = getInputs().get(inputId); - final Set inputUniqueKeys = getUpsertKeys(input); + final Set inputUniqueKeys = getUniqueKeys(input); if (inputUniqueKeys == null || inputUniqueKeys.isEmpty()) { return false; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeMultiJoinJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeMultiJoinJsonSerdeTest.java index 38454ac9fedfc..14e2489a06d5a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeMultiJoinJsonSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeMultiJoinJsonSerdeTest.java @@ -101,7 +101,7 @@ void testSerializedJsonStructure() throws IOException { // MultiJoin specific fields JsonSerdeTestUtil.assertThatJsonContains(jsonNode, "nodes", "0", "joinTypes"); JsonSerdeTestUtil.assertThatJsonContains(jsonNode, "nodes", "0", "joinAttributeMap"); - JsonSerdeTestUtil.assertThatJsonContains(jsonNode, "nodes", "0", "inputUpsertKeys"); + JsonSerdeTestUtil.assertThatJsonContains(jsonNode, "nodes", "0", "inputUniqueKeys"); JsonSerdeTestUtil.assertThatJsonContains(jsonNode, "nodes", "0", "joinConditions"); // Verify specific field values @@ -112,8 +112,8 @@ void testSerializedJsonStructure() throws IOException { assertThat(node.get("joinTypes")) .containsExactly(new TextNode("INNER"), new TextNode("INNER")); assertThat(node.get("joinAttributeMap").isObject()).isTrue(); - assertThat(node.get("inputUpsertKeys").isArray()).isTrue(); - assertThat(node.get("inputUpsertKeys")).hasSize(2); + assertThat(node.get("inputUniqueKeys").isArray()).isTrue(); + assertThat(node.get("inputUniqueKeys")).hasSize(2); assertThat(node.get("joinConditions").isArray()).isTrue(); assertThat(node.get("joinConditions")).hasSize(2); assertThat(node.get("inputProperties").isArray()).isTrue(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java index ef069feba055a..42db5806102e2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java @@ -39,6 +39,7 @@ public List programs() { MultiJoinTestPrograms.MULTI_JOIN_FOUR_WAY_NO_COMMON_JOIN_KEY, MultiJoinTestPrograms.MULTI_JOIN_THREE_WAY_LEFT_OUTER_JOIN_WITH_CTE, MultiJoinTestPrograms.MULTI_JOIN_LEFT_OUTER_WITH_NULL_KEYS, - MultiJoinTestPrograms.MULTI_JOIN_NULL_SAFE_JOIN_WITH_NULL_KEYS); + MultiJoinTestPrograms.MULTI_JOIN_NULL_SAFE_JOIN_WITH_NULL_KEYS, + MultiJoinTestPrograms.MULTI_JOIN_MIXED_CHANGELOG_MODES); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java index 38db1af79e15c..ffc47c62a20d0 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java @@ -545,7 +545,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") .consumedBeforeRestore( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -650,7 +650,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") .consumedValues( "+I[1, Gus, order0, 1, London]", "+I[1, Gus, order1, 1, London]", @@ -758,7 +758,7 @@ public class MultiJoinTestPrograms { "order_id STRING", "payment_id STRING", "location STRING") - .addOption("changelog-mode", "I,UA,UB,D") + .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") .consumedBeforeRestore( "+I[1, Gus, order0, payment1, London]", "+I[1, Gus, order1, payment1, London]", @@ -1042,4 +1042,64 @@ public class MultiJoinTestPrograms { + "FROM UsersNullSafe u " + "INNER JOIN OrdersNullSafe o ON u.user_id IS NOT DISTINCT FROM o.user_id") .build(); + + public static final TableTestProgram MULTI_JOIN_MIXED_CHANGELOG_MODES = + TableTestProgram.of( + "three-way-mixed-changelog-modes", + "three way join with mixed changelog modes and primary key configurations") + .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) + .setupTableSource( + SourceTestStep.newBuilder("AppendTable") + .addSchema("id STRING PRIMARY KEY NOT ENFORCED, val STRING") + .addOption("changelog-mode", "I") + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "append1"), + Row.ofKind(RowKind.INSERT, "2", "append2"), + Row.ofKind(RowKind.INSERT, "3", "append3")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("RetractTable") + .addSchema("ref_id STRING, data STRING") + .addOption("changelog-mode", "I,UA,UB,D") + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "retract1"), + Row.ofKind(RowKind.INSERT, "2", "retract2"), + Row.ofKind(RowKind.INSERT, "3", "retract3"), + Row.ofKind(RowKind.DELETE, "3", "retract3"), + Row.ofKind(RowKind.INSERT, "1", "retract1_new")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("UpsertTable") + .addSchema( + "key_id STRING PRIMARY KEY NOT ENFORCED, status STRING") + .addOption("changelog-mode", "I,UA,D") + .producedValues( + Row.ofKind(RowKind.INSERT, "1", "active"), + Row.ofKind(RowKind.INSERT, "2", "pending"), + Row.ofKind(RowKind.UPDATE_AFTER, "2", "active"), + Row.ofKind(RowKind.INSERT, "3", "inactive"), + Row.ofKind(RowKind.DELETE, "3", "inactive")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema( + "id STRING", + "val STRING", + "data STRING", + "status STRING") + .addOption("sink-changelog-mode-enforced", "I,UA,UB,D") + .consumedValues( + "+I[1, append1, retract1, active]", + "+I[2, append2, retract2, active]", + "+I[1, append1, retract1_new, active]", + "+I[3, append3, null, null]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT a.id, a.val, r.data, u.status " + + "FROM AppendTable a " + + "LEFT JOIN RetractTable r ON a.id = r.ref_id " + + "LEFT JOIN UpsertTable u ON a.id = u.key_id") + .build(); } diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json index 0a9fd86b2898c..8a415d1da8c1e 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json @@ -362,7 +362,7 @@ "rightFieldIndex" : 1 } ] }, - "inputUpsertKeys" : [ [ [ 1 ] ], [ ], [ ], [ ] ], + "inputUniqueKeys" : [ [ [ 1 ] ], [ [ 0 ] ], [ [ 1 ] ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/savepoint/_metadata index b176b457713c01326d696bd5357c1ba2722266db..061d0b08163df013497e898e81c8c18037fa96da 100644 GIT binary patch delta 1385 zcmbVLOK2295S^Jdt0H7K#!mu@CL7ko(4=Rkr+Zq|gdCIz3PvJ+KQlmf*CRd0#Yr`M?9kB@$mLNSFr)*F zRS+^sB4dA2-x)N#$;J$LCs z$FU4BMX_={L?TU*^h_m;bHW21hgw9*WO11@4{qWU%|huRPsb^@2#}zmK#a!B)xtAf zQ7~EjnzXE+STGNM*H>}q3xzaQzG-{pp-?i3h|&@>F10U2lIxky&4W9X6mx-NBwf!i zMp1;gFFZ4`hzpk~kI7U!u)gV`xwqX9Wu$z~kt?~22-g@%Eiej+qTJJjP!9Ks-;#s< zL$Pg z8fSGHO;2lfY$R4AzE$;S+>+&(&Au8Dd>Ugb6Sip;P&VQaa| z&cNl;4wx!;+Jo?++`6<*_KTxydTNP0t7`7mQD6e*gQqsDAfVSQ!nw75Hc1bNW3_LJW6~>G3Egx0(+qkK*`d! zo-uc(T=vM* z*T*RJQ!k{D5N7H}h`=buDMUW!$h=Y#dI*5_zxCDEORoMLeZ6)C8WBlSE~yVCr3oY` zO`$;2gM_9&V-iUj>$zP&twf(#aOx2rK^c?O$WM`xKc-Lw3=CNR}^$^)*;|ImMJ zops!oE*Zbead-TqzJZk$Naj;>a)k(?pu$KN2NZG?1`x$GMP8h692m)7-e4r_?QF}O z%9^8^Q3uAp&8`}*#_EUG0rlYB{N~ySm;fD>QP5VY4#y|uXd|)SvFW_Jemd`9efRXI zuG)w1sT+MG>fHV|J=p$+vvBjKqdptCrRyU*9Q{!wTK~r7bJVw&B%|E=A7b`QL7 z<92nkHl)7aeoqe`-|pyr_+eHbtzPPIgmNY~s-7vl#Y~)3eK6Z^W_%5R8S}+k`@pOg zd*cAtT#sA)x_+CmmNTo%tQXa`$%6WAve>?e&Y1M$>W2-_#FXVJs%ukS1c{DESBxvXv#UW=pV{y{YR+T1>@VuST(&>7v)NUB frrFgi%OabT8|-pj-JdC_FK3r@WzhQf*}a)ZS5|-I diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json index ecfa2f8e9f911..4b020388a6177 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json @@ -228,7 +228,7 @@ "rightFieldIndex" : 0 } ] }, - "inputUpsertKeys" : [ [ [ 1 ] ], [ ], [ ] ], + "inputUniqueKeys" : [ [ [ 1 ] ], [ [ 0 ] ], [ [ 1 ] ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", @@ -344,7 +344,7 @@ "rightFieldIndex" : 1 } ] }, - "inputUpsertKeys" : [ [ ], [ ] ], + "inputUniqueKeys" : [ [ ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/savepoint/_metadata index 5282c8c245b9a8763543df264e60ac4c650f8c75..f53211043d60e0457995ea123f9ad15aadae811a 100644 GIT binary patch delta 2003 zcmbVNeP~uy7{B+r57XA{-CSE!+_bs1JoBEfd+rr76`JohXYnpA%C+5l?!CHZ@7l*J zZb3MxVJvC$3_^o43PKA)Z&45t6AFR|)<&TOGr@mElA!)dx6XNA-_e>Ywg>Jx_uS{4 z=lt$@e$VfmxoCWF&KP+>Uv<9pkZ#=jb2O@dZj6o;8U<@d2Td`Mv*TGz7<4d(CLsu7 z$EFZ(?RAd?FLiY!`#|9NCbEDBvF8z&hCX$-nx>09n+L#gxC0317W+<<*nT#iDay z>iMPrk?ki5EHA6rg2Li_mecorYc1xb^%w52-sJ-fXp7{RYmn)3W6u)-v6$ilD=reHK zhz4@M(nZ&f1L9_KTGH}p)HxQ{|G~}U5wX1gS-W6A#c3hxQQmq*+cc;1(z!{9-|Nd_ zw)(1UjeJT~FMjCVAU4EvYq2&Y$C+jFC2d_je$XQwI5Qm74Ege(kv`tN@+moIewgg? zY%j}7(Iqh<8wDh7vRIW3s^%^*yWxQyYTPBTfBjLJT%665yd)xL8J{V=hJ}Ba1{?-&nm# z?lS+tvP-l5?5wCODXFBbS^=SLFQyLnb$}P7>~0>OY~SC1@Vg6t-un8DZJ)e*p}9sk z%2=kd%Kpk!)`Q{lD@*CU?^+x%VoUjv#UvG;G61pbQtq0Rg`TFBcgxD!WJJ>~xw@Vc zjXwjFh^uPZJg4S*ju?-*QNig7J(}i?h59bVcD+VP`kH@}Sd8`FkP~F)vk3CS03gVqly%IdbXgPxrsc>Z@JN6K?4J=Q z_T-mCb+5oS?jGi+tG@*`m#%;KSp83DpJ=@O>B;7@(G#o1mDJQSRH zE7k|D7n%V3TwX2#4P+*wk)|BI+)@avG=ka5``@2AG%JAW8| E12&xregFUf delta 2389 zcmb_dTWl0n7@nD>EiJmmrPT!t?v`q)aOjztGiS~;1_jbIO=xMft%Rm*=SsKqy4@-z zq_##HLv5iPAJi8in3$-5%{=G>Mljw&;w{1WLa{N%7mdb5qDg`E%+5|J786^UWX?Zx z=AZxi{oi@xE#c%x!tn?A!s%!SFNBls6#t$ub-Y*zpPL$mbk*Ze?1dT;B}J5ES=BTG zH4Jgpfc}i#*PHA~18Av+qyp&J&~PkERy3o^!o+lJ+XR}aO8^l?1%_ftK!U1fIEw7( zw#jkfYwq{`8wIKsyu*jwQ?cDVZD_elXPOVvKSv&k)b*sBhWc#Izj%S;==A!(*3>mL z)zsEJ$(1~PgMH!I9imgU7is;`OYSGDYVz~S*;b6gWOSwE|H{#dNIbV)ABh%Ox$R7b z+W1P#($h0-2gw%J0Jc;IAcSn7X^sYTY}g{UEK`>;9i6=4p094^h0Xn}renOYI+ktz~Ykkrj0(#~qP< zIFB-s=RNWq8x?L|zcTn9$PJ#7fBE{EOwPmXxaT2G4|hf%@~rP*)^kBDvVKc14@t?o512=0)4+j6h@+1$E2$?2K~%D-=c41fp0Ibs)!#Y>GGr8en(Be z2DfxqM+0kt49ZJuCVpB5319_D7I8!%A!0KUAOsK*8bFr~RT2$JH)O{hom|1MSK7D& zq$Ete%S?Z|zw~YQP*d{JH|JgpfAuRpd90kCuf8Xa+PHfWN^}o)rIVnO5v%g!^RG%b zFYM3%;B3ifvF28$xz>HGyO#u5jwMoWR@6KYXYohE!_c}0h|IEcxKqmjU&f(dZ6{~ zk=yKvKG+vRIIt7Qz#hHw(lGt_*yUwF70HGrOR5D7LJ&X@u~S1g2@oN~8e#{BXhHA1 zqA!z@TB(gIL{eg)cQ9qz$?FyGpWb{_oh<&Y@nUP;X_gev?h&4gVa+im)gZbod5cPH z=vb-(O;ItBrqk((L(NM+T1Bj49YVxlh)s6NNd`MwEgh($B|DCY5Y*_%2!^WnBo{3!ExAagXWLn53o^M5Gp%^wG kwuu17qC_0ckRfKhDLcaIkx$=^Jm}7?87On%i9dwD0rlyR8vpRvm-Bn#(4SNI;6H{fnyBbVXP`rbgGG4%B!FVpQ(ZV9$ z#8AP3J%JPb>_6W7T(z53`|{Dg_S^I2Wpu$fw#+O6MJ&_ep?lK%yPMlb@HJyK;G-6D z4y9tZs1{3dCV~&y0I63 z*Yn^0{pf8f!)Gj2ya9<(fr7~(841X+X04$V%)@$qwzmUZ{kxKM!FEwP$cm#xWQOr0 zgLjoFrr^|;ra5_Tnr6QlwtP4nHm9@WsXkO~b~zwa4iTub2XQ`v`fMnAHcT=$`x&k; GzGr{Csg!#F diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/plan/three-way-join-with-time-attributes-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/plan/three-way-join-with-time-attributes-with-restore.json index 03324337eccf9..3fabcaceda135 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/plan/three-way-join-with-time-attributes-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/plan/three-way-join-with-time-attributes-with-restore.json @@ -348,7 +348,7 @@ "rightFieldIndex" : 2 } ] }, - "inputUpsertKeys" : [ [ [ 0 ] ], [ ], [ ] ], + "inputUniqueKeys" : [ [ [ 0 ] ], [ [ 0 ] ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/savepoint/_metadata index c21172f3c7daa97e5a90e71959c1173999cbc5f9..f11caa837fe444830bc7f7d829291d09406cecf7 100644 GIT binary patch delta 898 zcmbW0K~EGh6vsQWu!tlE#l$5F?h;uIq-;CWX*)`ScmRY5YK-g(1lpNS!~-jWU_2Q0 zWa7omOS~ahFT}%mFnUoxfqnt?S>;YG@r&wUvLRQkS@fv~Z%rZ&v&`tr&;*336N!bDI*ctWE%)D>S)t&>wHb=ff{laQ?G%Z%*Tsl+y<)q~?;lP{VNg7V*e4D%3 zVz1}Im8sY8^WYeCT`q&3xdb=Q9{=BH;NlSUc66mn*Q##LMbq%!8H8`A&ZH}|^G?3U zHb4dVR4wF@W7Dg!RqZ|^vub9Y)(csLQtfoRP5P~)BFI{))wS7;J^A^pnAtyh!OBl| z#`{G?_v0h@Tw$fQqSwe^ipj`!TZwcYsgvSGE^91k5-)V|az zA;R;RV2TME3&mBKgfuovh7{`a=k^z9wuy)IY&&VB#w1n&W=42eCEUl6RK#bACnFxg z@WLPXUfXV-lEgu%?M`D)D1jOG60Eoha7c8lBVt!ckv+=M`RC13!iWc+Q8A7}YAG_N VI20bol7#jr_;v#0rNaxn#@o!JUv#5MT)SFJm_88e>Wg^ z@Gsk2!b0;{Cha=BwASV;Wd%O4&)w0G>>W6mNX7?-Qlq`eo>c!}cf6mccc;8)*CW}p zpjA>@bf#GRWKn)f!lFeHG*{s*?PzGVi_X{(tMz zZCUf|DK|a+Pjk6~&xU(>XW!n^zxsO1^s--kRQ8S>Dg<~-m&I=$PV)82<6{3&(k#?T zF@9^fw*1ZHx%FZ3&O}{ZkDv9UQlM5KCeSLtbHmjxl;bX=HJW<7i_h9A`qnbnF6zm@ACoh@qiG zqpBm!VcK5hD3`ZbpJK&=*tLMIKrw}?LV%&V5MXL!sJez>X_{|5;kuC5KBgA^0{o3L A;{X5v diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json index 72c4025056496..5105a4410d939 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json @@ -153,7 +153,7 @@ "rightFieldIndex" : 0 } ] }, - "inputUpsertKeys" : [ [ ], [ ], [ ] ], + "inputUniqueKeys" : [ [ ], [ ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/savepoint/_metadata index e3228781a06d4cce51c922fdb3064f156c5469e2..f0288325eeeb97c37f52f63977613c1e4dbbea34 100644 GIT binary patch delta 682 zcmbV}ze-#|6vprM&qlD&%4!W*B}5K8Gw00A84xiKuqz0XGS0t?DWZZR_y87G3iAf3 z62WS#6e&|}n<`d$fOKh1yjgE$xdSubFz4g^X1@9NrT+1`zPYPQx$bSflR_aY!-}0V zESL#taO%(V$?=DyGmz0TX&|Me4I(R%aNuM}AxaCWX%{l&AT+jtK*$!!>SV%rryecN zj_^);q$UDUa6kexc#NuriC{7-kLbo5FW$VG9T7MN6paAIQnnn!;FF1<8;L17jagvX zsUOPy;5+rI>F8f}`nA*FPrAO*>3*kIbv2Fm{S5IY6>Lnxf_DEe=1l{v69WpgSvS^X zq^er~)}nglK=jAA@~}TIs)h1)^m1*fJ$~KJ6MykBADzi|E9R%AG8y*EWVAW8{u-{8 zmEEQ1cGcOt^6Im0s@xB4`MLY)DfaytUKgFWHf0Z1%l2oDB4F=4Fe{SeWa6@2{~a&y H{h50N&+Dzu delta 745 zcmbV~ze`*}5XblV@VGOQbVA_nx`b;+clPa1NMjqglR!ve6L9Xu+cZQkS-S2!o&aOXYA5XHqc(wNImEp`# zDHtj(B}K{vUwLP@!`nBT(=D*fr3t|)MT>%$rROWQ6zGg&Adv^)f?BX#N^n)-16q_$ z>v}bt%r_NY-@SrC=6R_Dpeqr$@d~W+9s(LFP_r99Ay0P*AsrGA2ev+4I?L+kz5L0& zU&W*c$sXd@HSX4nEXEv^8zv_(_Kj;1c;QhH4 From 0bb5ef01f1b4684cf73371c866ad0f2c84628dca Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Fri, 19 Sep 2025 15:33:41 +0100 Subject: [PATCH 2/6] [FLINK-38379][table] Adjust `visitMultiJoin` in `RelTimeIndicatorConverter` to use fields and not rows --- .claude/CLAUDE.local.md | 8 ++ .../calcite/RelTimeIndicatorConverter.java | 4 +- .../exec/stream/MultiJoinSemanticTests.java | 4 +- .../exec/stream/MultiJoinTestPrograms.java | 97 +++++++++++++++++++ 4 files changed, 111 insertions(+), 2 deletions(-) create mode 100644 .claude/CLAUDE.local.md diff --git a/.claude/CLAUDE.local.md b/.claude/CLAUDE.local.md new file mode 100644 index 0000000000000..8d9ff2b2c095e --- /dev/null +++ b/.claude/CLAUDE.local.md @@ -0,0 +1,8 @@ +- You're a respected open source apache flink maintainer. + +- Test your changes with one appropriate existing test using this format just to check if we have no compilation errors "cd /Users/gdemorais/qdev/flink2 && ./mvnw test -Dtest="MultiJoinTest#testTwoWayJoinWithUnion" -pl flink-table/flink-table-planner -q -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true" +- If you cannot run your tests because they get stuck downloading dependencies. You can run "./mvnw clean install -T1C -DskipTests -Pfast -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true -DskipITs=true -Dmaven.javadoc.skip=true -Djapicmp.skip=true -Pskip-webui-build -T4 -fn" to build first. Just do this once and if necessary because it takes over 5 minutes! +- If you have to install only one module, also use all the available flags to speed up the process. E.g. "./mvnw install -pl flink-table/flink-table-common -T1C -DskipTests -Pfast -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true -DskipITs=true -Dmaven.javadoc.skip=true -Djapicmp.skip=true -Pskip-webui-build -T4" + +- We always want to keep the things we do in "eval" for functions and in the processRecord methods in operators to a minimum since this is a hot path for flink which is executed millions of times. Always do what you can in the constructor, intialize things else where. + diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java index cffe954de322a..305296e5aff07 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java @@ -580,7 +580,9 @@ private RelNode visitMultiJoin(FlinkLogicalMultiJoin multiJoin) { .collect(Collectors.toList()); final List allFields = - newInputs.stream().map(RelNode::getRowType).collect(Collectors.toList()); + newInputs.stream() + .flatMap(input -> RelOptUtil.getFieldTypeList(input.getRowType()).stream()) + .collect(Collectors.toList()); RexTimeIndicatorMaterializer materializer = new RexTimeIndicatorMaterializer(allFields); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java index 42db5806102e2..fb24e92e1d736 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java @@ -40,6 +40,8 @@ public List programs() { MultiJoinTestPrograms.MULTI_JOIN_THREE_WAY_LEFT_OUTER_JOIN_WITH_CTE, MultiJoinTestPrograms.MULTI_JOIN_LEFT_OUTER_WITH_NULL_KEYS, MultiJoinTestPrograms.MULTI_JOIN_NULL_SAFE_JOIN_WITH_NULL_KEYS, - MultiJoinTestPrograms.MULTI_JOIN_MIXED_CHANGELOG_MODES); + MultiJoinTestPrograms.MULTI_JOIN_MIXED_CHANGELOG_MODES, + MultiJoinTestPrograms + .MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java index ffc47c62a20d0..3e8e27e9c305f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java @@ -1102,4 +1102,101 @@ public class MultiJoinTestPrograms { + "LEFT JOIN RetractTable r ON a.id = r.ref_id " + "LEFT JOIN UpsertTable u ON a.id = u.key_id") .build(); + + public static final TableTestProgram + MULTI_JOIN_WITH_TIME_ATTRIBUTES_IN_CONDITIONS_MATERIALIZATION = + TableTestProgram.of( + "three-way-join-with-time-attributes-in-join-conditions", + "A query from the nexmark benchmark: " + + "auction and bid aggregation with time-based filtering") + .setupTableSource( + SourceTestStep.newBuilder("auctions") + .addSchema( + "id BIGINT PRIMARY KEY NOT ENFORCED", + "category STRING", + "auctionTimestamp STRING", + "expiresTimestamp STRING", + "auctionDateTime AS TO_TIMESTAMP(auctionTimestamp)", + "expires AS TO_TIMESTAMP(expiresTimestamp)", + "WATERMARK FOR auctionDateTime AS auctionDateTime - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I") + .producedValues( + Row.ofKind( + RowKind.INSERT, + 1L, + "Electronics", + "2024-01-01 12:00:00", + "2024-01-01 12:30:00"), + Row.ofKind( + RowKind.INSERT, + 3L, + "Electronics", + "2024-01-01 12:10:00", + "2024-01-01 12:40:00"), + Row.ofKind( + RowKind.INSERT, + 2L, + "Books", + "2024-01-01 12:05:00", + "2024-01-01 12:35:00")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("bids") + .addSchema( + "auction BIGINT", + "price DOUBLE", + "bidTimestamp STRING", + "bidDateTime AS TO_TIMESTAMP(bidTimestamp)", + "WATERMARK FOR bidDateTime AS bidDateTime - INTERVAL '1' SECOND") + .addOption("changelog-mode", "I") + .producedValues( + Row.ofKind( + RowKind.INSERT, + 1L, + 12.0, + "2024-01-01 12:15:00"), + Row.ofKind( + RowKind.INSERT, + 1L, + 15.0, + "2024-01-01 12:20:00"), + Row.ofKind( + RowKind.INSERT, + 2L, + 25.0, + "2024-01-01 12:25:00"), + Row.ofKind( + RowKind.INSERT, + 3L, + 18.0, + "2024-01-01 12:30:00"), + Row.ofKind( + RowKind.INSERT, + 1L, + 20.0, + "2024-01-01 12:45:00")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink") + .addSchema("category STRING", "avg_final DOUBLE") + .consumedValues( + // Electronics: AVG(MAX(12.0, 15.0), MAX(18.0)) + // = + // AVG(15.0, 18.0) = 16.5 + "+I[Electronics, 16.5]", + // Books: MAX(25.0) = 25.0, AVG(25.0) = 25.0 + "+I[Books, 25.0]") + .testMaterializedData() + .build()) + .runSql( + "INSERT INTO sink " + + "SELECT Q.category, AVG(Q.final) " + + "FROM ( " + + " SELECT MAX(B.price) AS final, A.category " + + " FROM auctions A, bids B " + + " WHERE A.id = B.auction AND B.bidDateTime BETWEEN A.auctionDateTime AND A.expires " + + " GROUP BY A.id, A.category " + + ") Q " + + "GROUP BY Q.category") + .build(); } From 9e4b4af4c16f62e4cae1dfa21cc291d086022d40 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Fri, 24 Oct 2025 11:57:15 +0200 Subject: [PATCH 3/6] [FLINK-38554][table] Fix rowCount cost for FlinkLogicalMultiJoin --- .../nodes/logical/FlinkLogicalMultiJoin.java | 2 +- .../plan/stream/sql/MultiJoinTest.java | 39 ++++++++++++ .../planner/plan/stream/sql/MultiJoinTest.xml | 62 +++++++++++++++++++ 3 files changed, 102 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMultiJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMultiJoin.java index 5429629f9919b..21bb14c73a004 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMultiJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMultiJoin.java @@ -153,7 +153,7 @@ public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadata final Double averageRowSize = mq.getAverageRowSize(input); final double dAverageRowSize = averageRowSize == null ? 100.0 : averageRowSize; - rowCount *= inputRowCount; + rowCount += inputRowCount; cpu += inputRowCount; io += inputRowCount * dAverageRowSize; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java index a6a488d3db375..8121d763d8419 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java @@ -171,6 +171,45 @@ void testThreeWayLeftOuterJoinExecPlan() { + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); } + @Test + void testTwoWayJoinWithUnion() { + util.tableEnv() + .executeSql( + "CREATE TABLE Orders2 (" + + " order_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id_1 STRING," + + " product STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')"); + + util.verifyRelPlan( + "WITH OrdersUnion as (" + + "SELECT * FROM Orders " + + "UNION ALL " + + "SELECT * FROM Orders2" + + ") " + + "SELECT * FROM OrdersUnion o " + + "LEFT JOIN Users u " + + "ON o.user_id_1 = u.user_id_0"); + } + + @Test + void testTwoWayJoinWithRank() { + util.getTableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true); + + util.verifyRelPlan( + "WITH JoinedEvents as (" + + "SELECT e1.id as id, e1.val, e1.rowtime as `rowtime`, e2.price " + + "FROM EventTable1 e1 " + + "JOIN EventTable2 e2 ON e1.id = e2.id) " + + "SELECT id, val, `rowtime` FROM (" + + "SELECT *, " + + "ROW_NUMBER() OVER (PARTITION BY id ORDER BY `rowtime` DESC) as ts " + + "FROM JoinedEvents) " + + "WHERE ts = 1"); + } + @Test void testFourWayComplexJoinRelPlan() { util.verifyRelPlan( diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index ba05d7b677837..7df8d36563af3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -703,6 +703,68 @@ Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CA +- Exchange(distribution=[hash[user_id_2]]) +- Calc(select=[payment_id, price, user_id_2], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +]]> + + + + + + + + + + + + + + + + + + + + + + From 18023c34f8e46a570d0cc375f57a26ea3e9fa160 Mon Sep 17 00:00:00 2001 From: Stepan Stepanishchev <116617250+SteveStevenpoor@users.noreply.github.com> Date: Tue, 18 Nov 2025 16:05:17 +0700 Subject: [PATCH 4/6] [FLINK-38576][table] Align commonJoinKey in MultiJoin for logical and physical rules --- .claude/CLAUDE.local.md | 8 - .../table/utils/NoCommonJoinKeyException.java | 40 + .../rules/logical/JoinToMultiJoinRule.java | 170 +- .../stream/StreamPhysicalMultiJoinRule.java | 114 +- .../planner/plan/utils/MultiJoinUtil.java | 155 + .../plan/stream/sql/MultiJoinTest.java | 1521 +++++++++- .../planner/plan/stream/sql/MultiJoinTest.xml | 2589 ++++++++++++++--- .../AttributeBasedJoinKeyExtractor.java | 3 +- 8 files changed, 3816 insertions(+), 784 deletions(-) delete mode 100644 .claude/CLAUDE.local.md create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java diff --git a/.claude/CLAUDE.local.md b/.claude/CLAUDE.local.md deleted file mode 100644 index 8d9ff2b2c095e..0000000000000 --- a/.claude/CLAUDE.local.md +++ /dev/null @@ -1,8 +0,0 @@ -- You're a respected open source apache flink maintainer. - -- Test your changes with one appropriate existing test using this format just to check if we have no compilation errors "cd /Users/gdemorais/qdev/flink2 && ./mvnw test -Dtest="MultiJoinTest#testTwoWayJoinWithUnion" -pl flink-table/flink-table-planner -q -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true" -- If you cannot run your tests because they get stuck downloading dependencies. You can run "./mvnw clean install -T1C -DskipTests -Pfast -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true -DskipITs=true -Dmaven.javadoc.skip=true -Djapicmp.skip=true -Pskip-webui-build -T4 -fn" to build first. Just do this once and if necessary because it takes over 5 minutes! -- If you have to install only one module, also use all the available flags to speed up the process. E.g. "./mvnw install -pl flink-table/flink-table-common -T1C -DskipTests -Pfast -Dcheckstyle.skip -Drat.skip -Dscalastyle.skip -Denforcer.skip=true -Pgenerate-config-docs -Dspotless.check.skip=true -DskipITs=true -Dmaven.javadoc.skip=true -Djapicmp.skip=true -Pskip-webui-build -T4" - -- We always want to keep the things we do in "eval" for functions and in the processRecord methods in operators to a minimum since this is a hot path for flink which is executed millions of times. Always do what you can in the constructor, intialize things else where. - diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java new file mode 100644 index 0000000000000..21eb273b91171 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/NoCommonJoinKeyException.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.FlinkRuntimeException; + +/** Thrown when a MultiJoin node has no common join key. */ +@Internal +public class NoCommonJoinKeyException extends FlinkRuntimeException { + private static final long serialVersionUID = 1L; + + public NoCommonJoinKeyException(String message) { + super(message); + } + + public NoCommonJoinKeyException(String message, Throwable cause) { + super(message, cause); + } + + public NoCommonJoinKeyException(Throwable cause) { + super(cause); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java index 762c16b784da6..922ff718ec78e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java @@ -18,14 +18,19 @@ package org.apache.flink.table.planner.plan.rules.logical; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.hint.StateTtlHint; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; import org.apache.flink.table.planner.plan.utils.IntervalJoinUtil; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; +import org.apache.flink.table.runtime.operators.join.stream.keyselector.JoinKeyExtractor; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.NoCommonJoinKeyException; import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.RelRule; import org.apache.calcite.plan.hep.HepRelVertex; @@ -38,17 +43,15 @@ import org.apache.calcite.rel.core.TableFunctionScan; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.core.Values; +import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalSnapshot; -import org.apache.calcite.rel.metadata.RelColumnOrigin; -import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.rel.rules.FilterMultiJoinMergeRule; import org.apache.calcite.rel.rules.MultiJoin; import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule; import org.apache.calcite.rel.rules.TransformationRule; import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexUtil; @@ -62,11 +65,15 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.table.planner.hint.StateTtlHint.STATE_TTL; +import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap; /** * Flink Planner rule to flatten a tree of {@link Join}s into a single {@link MultiJoin} with N @@ -462,134 +469,45 @@ private boolean canCombine(RelNode input, Join origJoin) { /** * Checks if original join and child multi-join have common join keys to decide if we can merge - * them into a single MultiJoin with one more input. + * them into a single MultiJoin with one more input. The method uses {@link + * AttributeBasedJoinKeyExtractor} to try to create valid common join key extractors. * * @param origJoin original Join * @param otherJoin child MultiJoin * @return true if original Join and child multi-join have at least one common JoinKey */ private boolean haveCommonJoinKey(Join origJoin, MultiJoin otherJoin) { - Set origJoinKeys = getJoinKeys(origJoin); - Set otherJoinKeys = getJoinKeys(otherJoin); - - origJoinKeys.retainAll(otherJoinKeys); - - return !origJoinKeys.isEmpty(); - } - - /** - * Returns a set of join keys as strings following this format [table_name.field_name]. - * - * @param join Join or MultiJoin node - * @return set of all the join keys (keys from join conditions) - */ - public Set getJoinKeys(RelNode join) { - Set joinKeys = new HashSet<>(); - List conditions = Collections.emptyList(); - List inputs = join.getInputs(); - - if (join instanceof Join) { - conditions = collectConjunctions(((Join) join).getCondition()); - } else if (join instanceof MultiJoin) { - conditions = - ((MultiJoin) join) - .getOuterJoinConditions().stream() - .flatMap(cond -> collectConjunctions(cond).stream()) - .collect(Collectors.toList()); - } - - RelMetadataQuery mq = join.getCluster().getMetadataQuery(); - - for (RexCall condition : conditions) { - for (RexNode operand : condition.getOperands()) { - if (operand instanceof RexInputRef) { - addJoinKeysByOperand((RexInputRef) operand, inputs, mq, joinKeys); - } - } - } - - return joinKeys; - } - - /** - * Retrieves conjunctions from joinCondition. - * - * @param joinCondition join condition - * @return List of RexCalls representing conditions - */ - private List collectConjunctions(RexNode joinCondition) { - return RelOptUtil.conjunctions(joinCondition).stream() - .map(rexNode -> (RexCall) rexNode) - .collect(Collectors.toList()); - } - - /** - * Appends join key's string representation to the set of join keys. - * - * @param ref input ref to the operand - * @param inputs List of node's inputs - * @param mq RelMetadataQuery needed to retrieve column origins - * @param joinKeys Set of join keys to be added - */ - private void addJoinKeysByOperand( - RexInputRef ref, List inputs, RelMetadataQuery mq, Set joinKeys) { - int inputRefIndex = ref.getIndex(); - Tuple2 targetInputAndIdx = getTargetInputAndIdx(inputRefIndex, inputs); - RelNode targetInput = targetInputAndIdx.f0; - int idxInTargetInput = targetInputAndIdx.f1; - - Set origins = mq.getColumnOrigins(targetInput, idxInTargetInput); - if (origins != null) { - for (RelColumnOrigin origin : origins) { - RelOptTable originTable = origin.getOriginTable(); - List qualifiedName = originTable.getQualifiedName(); - String fieldName = - originTable - .getRowType() - .getFieldList() - .get(origin.getOriginColumnOrdinal()) - .getName(); - joinKeys.add(qualifiedName.get(qualifiedName.size() - 1) + "." + fieldName); - } + final List combinedJoinInputs = + Stream.concat(otherJoin.getInputs().stream(), Stream.of(origJoin.getRight())) + .collect(Collectors.toUnmodifiableList()); + + final List combinedInputTypes = + combinedJoinInputs.stream() + .map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType())) + .collect(Collectors.toUnmodifiableList()); + + final List combinedJoinConditions = + Stream.concat( + otherJoin.getOuterJoinConditions().stream(), + List.of(origJoin.getCondition()).stream()) + .collect(Collectors.toUnmodifiableList()); + + final Map> + joinAttributeMap = + createJoinAttributeMap(combinedJoinInputs, combinedJoinConditions); + + boolean haveCommonJoinKey = false; + try { + // we probe to instantiate AttributeBasedJoinKeyExtractor's constructor to check whether + // it's possible to initialize common join key structures + final JoinKeyExtractor keyExtractor = + new AttributeBasedJoinKeyExtractor(joinAttributeMap, combinedInputTypes); + haveCommonJoinKey = keyExtractor.getCommonJoinKeyIndices(0).length > 0; + } catch (NoCommonJoinKeyException ignored) { + // failed to instantiate common join key structures => no common join key } - } - /** - * Get real table that contains needed input ref (join key). - * - * @param inputRefIndex index of the required field - * @param inputs inputs of the node - * @return target input + idx of the required field as target input's - */ - private Tuple2 getTargetInputAndIdx(int inputRefIndex, List inputs) { - RelNode targetInput = null; - int idxInTargetInput = 0; - int inputFieldEnd = 0; - for (RelNode input : inputs) { - inputFieldEnd += input.getRowType().getFieldCount(); - if (inputRefIndex < inputFieldEnd) { - targetInput = input; - int targetInputStartIdx = inputFieldEnd - input.getRowType().getFieldCount(); - idxInTargetInput = inputRefIndex - targetInputStartIdx; - break; - } - } - - targetInput = - (targetInput instanceof HepRelVertex) - ? ((HepRelVertex) targetInput).getCurrentRel() - : targetInput; - - assert targetInput != null; - - if (targetInput instanceof TableScan - || targetInput instanceof Values - || targetInput instanceof TableFunctionScan - || targetInput.getInputs().isEmpty()) { - return new Tuple2<>(targetInput, idxInTargetInput); - } else { - return getTargetInputAndIdx(idxInTargetInput, targetInput.getInputs()); - } + return haveCommonJoinKey; } /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java index e59ea9df9b9d5..da3b99d669a73 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalMultiJoinRule.java @@ -32,18 +32,14 @@ import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.SqlKind; -import org.checkerframework.checker.nullness.qual.Nullable; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap; + /** Rule that converts {@link FlinkLogicalMultiJoin} to {@link StreamPhysicalMultiJoin}. */ public class StreamPhysicalMultiJoinRule extends ConverterRule { public static final RelOptRule INSTANCE = new StreamPhysicalMultiJoinRule(); @@ -61,7 +57,7 @@ private StreamPhysicalMultiJoinRule() { public RelNode convert(final RelNode rel) { final FlinkLogicalMultiJoin multiJoin = (FlinkLogicalMultiJoin) rel; final Map> joinAttributeMap = - createJoinAttributeMap(multiJoin); + createJoinAttributeMap(multiJoin.getInputs(), multiJoin.getJoinConditions()); final List inputRowTypes = multiJoin.getInputs().stream() .map(i -> FlinkTypeFactory.toLogicalRowType(i.getRowType())) @@ -117,108 +113,4 @@ private RelTraitSet createInputTraitSet( return inputTraitSet; } - - private Map> createJoinAttributeMap( - final FlinkLogicalMultiJoin multiJoin) { - final Map> joinAttributeMap = new HashMap<>(); - final List inputFieldCounts = - multiJoin.getInputs().stream() - .map(input -> input.getRowType().getFieldCount()) - .collect(Collectors.toList()); - - final List inputOffsets = new ArrayList<>(); - int currentOffset = 0; - for (final Integer count : inputFieldCounts) { - inputOffsets.add(currentOffset); - currentOffset += count; - } - - final List joinConditions = multiJoin.getJoinConditions(); - for (final RexNode condition : joinConditions) { - extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap); - } - return joinAttributeMap; - } - - private void extractEqualityConditions( - final RexNode condition, - final List inputOffsets, - final List inputFieldCounts, - final Map> joinAttributeMap) { - if (!(condition instanceof RexCall)) { - return; - } - - final RexCall call = (RexCall) condition; - final SqlKind kind = call.getOperator().getKind(); - - if (kind != SqlKind.EQUALS) { - for (final RexNode operand : call.getOperands()) { - extractEqualityConditions( - operand, inputOffsets, inputFieldCounts, joinAttributeMap); - } - return; - } - - if (call.getOperands().size() != 2) { - return; - } - - final RexNode op1 = call.getOperands().get(0); - final RexNode op2 = call.getOperands().get(1); - - if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) { - return; - } - - final InputRef inputRef1 = - findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts); - final InputRef inputRef2 = - findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts); - - if (inputRef1 == null || inputRef2 == null) { - return; - } - - final InputRef leftRef; - final InputRef rightRef; - if (inputRef1.inputIndex < inputRef2.inputIndex) { - leftRef = inputRef1; - rightRef = inputRef2; - } else { - leftRef = inputRef2; - rightRef = inputRef1; - } - - final ConditionAttributeRef attrRef = - new ConditionAttributeRef( - leftRef.inputIndex, - leftRef.attributeIndex, - rightRef.inputIndex, - rightRef.attributeIndex); - joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef); - } - - private @Nullable InputRef findInputRef( - final int fieldIndex, - final List inputOffsets, - final List inputFieldCounts) { - for (int i = 0; i < inputOffsets.size(); i++) { - final int offset = inputOffsets.get(i); - if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) { - return new InputRef(i, fieldIndex - offset); - } - } - return null; - } - - private static final class InputRef { - private final int inputIndex; - private final int attributeIndex; - - private InputRef(final int inputIndex, final int attributeIndex) { - this.inputIndex = inputIndex; - this.attributeIndex = attributeIndex; - } - } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java new file mode 100644 index 0000000000000..83bf9be50c1a3 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/MultiJoinUtil.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils; + +import org.apache.flink.table.runtime.operators.join.stream.keyselector.AttributeBasedJoinKeyExtractor; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class MultiJoinUtil { + public static Map> + createJoinAttributeMap( + List joinInputs, List joinConditions) { + final Map> + joinAttributeMap = new HashMap<>(); + final List inputFieldCounts = + joinInputs.stream() + .map(input -> input.getRowType().getFieldCount()) + .collect(Collectors.toList()); + + final List inputOffsets = new ArrayList<>(); + int currentOffset = 0; + for (final Integer count : inputFieldCounts) { + inputOffsets.add(currentOffset); + currentOffset += count; + } + + for (final RexNode condition : joinConditions) { + extractEqualityConditions(condition, inputOffsets, inputFieldCounts, joinAttributeMap); + } + return joinAttributeMap; + } + + private static void extractEqualityConditions( + final RexNode condition, + final List inputOffsets, + final List inputFieldCounts, + final Map> + joinAttributeMap) { + if (!(condition instanceof RexCall)) { + return; + } + + final RexCall call = (RexCall) condition; + final SqlKind kind = call.getOperator().getKind(); + + if (kind != SqlKind.EQUALS) { + for (final RexNode operand : call.getOperands()) { + extractEqualityConditions( + operand, inputOffsets, inputFieldCounts, joinAttributeMap); + } + return; + } + + if (call.getOperands().size() != 2) { + return; + } + + final RexNode op1 = call.getOperands().get(0); + final RexNode op2 = call.getOperands().get(1); + + if (!(op1 instanceof RexInputRef) || !(op2 instanceof RexInputRef)) { + return; + } + + final InputRef inputRef1 = + findInputRef(((RexInputRef) op1).getIndex(), inputOffsets, inputFieldCounts); + final InputRef inputRef2 = + findInputRef(((RexInputRef) op2).getIndex(), inputOffsets, inputFieldCounts); + + if (inputRef1 == null || inputRef2 == null) { + return; + } + + final InputRef leftRef; + final InputRef rightRef; + if (inputRef1.inputIndex < inputRef2.inputIndex) { + leftRef = inputRef1; + rightRef = inputRef2; + } else { + leftRef = inputRef2; + rightRef = inputRef1; + } + + // Special case for input 0: + // Since we are building attribute references that do left -> right index, + // we need a special base case for input 0 which has no input to the left. + // So we do {-1, -1} -> {0, attributeIndex} + if (leftRef.inputIndex == 0) { + final AttributeBasedJoinKeyExtractor.ConditionAttributeRef firstAttrRef = + new AttributeBasedJoinKeyExtractor.ConditionAttributeRef( + -1, -1, leftRef.inputIndex, leftRef.attributeIndex); + joinAttributeMap + .computeIfAbsent(leftRef.inputIndex, k -> new ArrayList<>()) + .add(firstAttrRef); + } + + final AttributeBasedJoinKeyExtractor.ConditionAttributeRef attrRef = + new AttributeBasedJoinKeyExtractor.ConditionAttributeRef( + leftRef.inputIndex, + leftRef.attributeIndex, + rightRef.inputIndex, + rightRef.attributeIndex); + joinAttributeMap.computeIfAbsent(rightRef.inputIndex, k -> new ArrayList<>()).add(attrRef); + } + + private static @Nullable InputRef findInputRef( + final int fieldIndex, + final List inputOffsets, + final List inputFieldCounts) { + for (int i = 0; i < inputOffsets.size(); i++) { + final int offset = inputOffsets.get(i); + if (fieldIndex >= offset && fieldIndex < offset + inputFieldCounts.get(i)) { + return new InputRef(i, fieldIndex - offset); + } + } + return null; + } + + private static final class InputRef { + private final int inputIndex; + private final int attributeIndex; + + private InputRef(final int inputIndex, final int attributeIndex) { + this.inputIndex = inputIndex; + this.attributeIndex = attributeIndex; + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java index 8121d763d8419..81cc1a10ef9e3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.java @@ -18,18 +18,26 @@ package org.apache.flink.table.planner.plan.stream.sql; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.StatementSet; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.utils.PlanKind; +import org.apache.flink.table.planner.utils.StreamTableTestUtil; import org.apache.flink.table.planner.utils.TableTestBase; -import org.apache.flink.table.planner.utils.TableTestUtil; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import scala.Enumeration; + +import static scala.runtime.BoxedUnit.UNIT; + /** Tests for multi-join plans. */ public class MultiJoinTest extends TableTestBase { - private TableTestUtil util; + private StreamTableTestUtil util; @BeforeEach void setup() { @@ -43,7 +51,7 @@ void setup() { util.tableEnv() .executeSql( "CREATE TABLE Users (" - + " user_id_0 STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING PRIMARY KEY NOT ENFORCED," + " name STRING," + " cash INT" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); @@ -52,7 +60,7 @@ void setup() { .executeSql( "CREATE TABLE Orders (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," + + " user_id STRING," + " product STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')"); @@ -61,16 +69,26 @@ void setup() { "CREATE TABLE Payments (" + " payment_id STRING PRIMARY KEY NOT ENFORCED," + " price INT," - + " user_id_2 STRING" + + " user_id STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); util.tableEnv() .executeSql( "CREATE TABLE Shipments (" + " location STRING," - + " user_id_3 STRING" + + " user_id STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,UB,D')"); + util.tableEnv() + .executeSql( + "CREATE TABLE Detail (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " description STRING," + + " user_id STRING," + + " data STRING," + + " `timestamp` BIGINT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); + // Tables for testing temporal join exclusion util.tableEnv() .executeSql( @@ -94,8 +112,8 @@ void setup() { "CREATE TABLE EventTable1 (" + " id STRING," + " val INT," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND" + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); util.tableEnv() @@ -103,15 +121,15 @@ void setup() { "CREATE TABLE EventTable2 (" + " id STRING," + " price DOUBLE," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND" + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime` - INTERVAL '5' SECOND" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); // Tables for testing time attribute materialization in multi-join util.tableEnv() .executeSql( "CREATE TABLE UsersWithProctime (" - + " user_id_0 STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING PRIMARY KEY NOT ENFORCED," + " name STRING," + " proctime AS PROCTIME()" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); @@ -120,55 +138,152 @@ void setup() { .executeSql( "CREATE TABLE OrdersWithRowtime (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," - + " rowtime TIMESTAMP(3)," - + " WATERMARK FOR rowtime AS rowtime" + + " user_id STRING," + + " `$rowtime` TIMESTAMP(3)," + + " WATERMARK FOR `$rowtime` AS `$rowtime`" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I')"); + // Tables for testing upsert key preservation + util.tableEnv() + .executeSql( + "CREATE TABLE UsersPK (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " name STRING," + + " region_id INT," + + " description STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE OrdersPK (" + + " order_id STRING NOT NULL," + + " user_id STRING NOT NULL," + + " product STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (order_id, user_id) NOT ENFORCED" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE PaymentsPK (" + + " payment_id STRING NOT NULL," + + " user_id STRING NOT NULL," + + " price INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (payment_id, user_id) NOT ENFORCED" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE AddressPK (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " location STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); } @Test void testThreeWayInnerJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test - void testThreeWayInnerJoinNoCommonJoinKeyRelPlan() { + @Tag("no-common-join-key") + void testThreeWayInnerJoinRelPlanNoCommonJoinKey() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.cash = p.price"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.cash = p.price"); } @Test void testThreeWayInnerJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); + } + + @Test + void testThreeWayInnerJoinWithTttlHints() { + util.verifyRelPlan( + "\nSELECT\n" + + " /*+ STATE_TTL(u='1d', o='2d', p='1h') */\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); + } + + @Test + void testThreeWayInnerJoinWithSingleTttlHint() { + util.verifyRelPlan( + "\nSELECT\n" + + " /*+ STaTE_tTL(o='2d') */\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test @@ -177,154 +292,241 @@ void testTwoWayJoinWithUnion() { .executeSql( "CREATE TABLE Orders2 (" + " order_id STRING PRIMARY KEY NOT ENFORCED," - + " user_id_1 STRING," + + " user_id STRING," + " product STRING" + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,D')"); util.verifyRelPlan( - "WITH OrdersUnion as (" - + "SELECT * FROM Orders " - + "UNION ALL " - + "SELECT * FROM Orders2" - + ") " - + "SELECT * FROM OrdersUnion o " - + "LEFT JOIN Users u " - + "ON o.user_id_1 = u.user_id_0"); + "\nWITH OrdersUnion as (\n" + + "SELECT * FROM Orders\n" + + "UNION ALL\n" + + "SELECT * FROM Orders2\n" + + ")\n" + + "SELECT * FROM OrdersUnion o\n" + + "LEFT JOIN Users u\n" + + " ON o.user_id = u.user_id"); } @Test void testTwoWayJoinWithRank() { - util.getTableEnv() - .getConfig() - .set(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true); - - util.verifyRelPlan( - "WITH JoinedEvents as (" - + "SELECT e1.id as id, e1.val, e1.rowtime as `rowtime`, e2.price " - + "FROM EventTable1 e1 " - + "JOIN EventTable2 e2 ON e1.id = e2.id) " - + "SELECT id, val, `rowtime` FROM (" - + "SELECT *, " - + "ROW_NUMBER() OVER (PARTITION BY id ORDER BY `rowtime` DESC) as ts " - + "FROM JoinedEvents) " + util.verifyRelPlan( + "\nWITH JoinedEvents as (\n" + + "SELECT\n" + + " e1.id as id,\n" + + " e1.val,\n" + + " e1.`$rowtime` as `$rowtime`,\n" + + " e2.price\n" + + "FROM EventTable1 e1\n" + + "JOIN EventTable2 e2\n" + + " ON e1.id = e2.id)\n" + + "SELECT\n" + + " id,\n" + + " val,\n" + + " `$rowtime`\n" + + "FROM (\n" + + " SELECT\n" + + " *,\n" + + " ROW_NUMBER() OVER (PARTITION BY id ORDER BY `$rowtime` DESC) as ts\n" + + " FROM JoinedEvents)\n" + "WHERE ts = 1"); } @Test void testFourWayComplexJoinRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test - void testThreeWayJoinNoJoinKeyExecPlan() { + @Tag("no-common-join-key") + void testThreeWayJoinExecPlanNoCommonJoinKey() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON TRUE " - + "INNER JOIN Payments p ON TRUE "); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o ON TRUE\n" + + "INNER JOIN Payments p ON TRUE"); } @Test - void testFourWayJoinNoCommonJoinKeyRelPlan() { + @Tag("no-common-join-key") + void testFourWayJoinRelPlanNoCommonJoinKey() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 " - + "LEFT JOIN Shipments s ON p.payment_id = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN LookupTable\n" + + " ON u.name = LookupTable.name\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON o.user_id = s.user_id"); } @Test void testFourWayComplexJoinExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test void testThreeWayInnerJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testThreeWayLeftOuterJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id"); } @Test void testFourWayComplexJoinExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN Payments p ON u.user_id_0 = p.user_id_2 AND (u.cash >= p.price OR p.price < 0) " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND (u.cash >= p.price OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } @Test void testTemporalJoinExcludedFromMultiJoin() { // Temporal joins should remain as lookup joins, not be merged into MultiJoin util.verifyRelPlan( - "SELECT s.user_id, s.amount, l.name, l.age " - + "FROM StreamTable s " - + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l " - + "ON s.user_id = l.id"); + "\nSELECT\n" + + " s.user_id,\n" + + " s.amount,\n" + + " l.name,\n" + + " l.age\n" + + "FROM StreamTable s\n" + + "JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n" + + " ON s.user_id = l.id"); } @Test void testIntervalJoinExcludedFromMultiJoin() { // Interval joins (event-time and processing-time) should remain as interval joins util.verifyRelPlan( - "SELECT e1.id, e1.val, e2.price " - + "FROM EventTable1 e1 " - + "JOIN EventTable2 e2 ON e1.id = e2.id " - + "AND e1.rowtime BETWEEN e2.rowtime - INTERVAL '1' MINUTE " - + "AND e2.rowtime + INTERVAL '1' MINUTE"); + "\nSELECT\n" + + " e1.id,\n" + + " e1.val,\n" + + " e2.price\n" + + "FROM EventTable1 e1\n" + + "JOIN EventTable2 e2\n" + + " ON e1.id = e2.id\n" + + " AND e1.`$rowtime` BETWEEN e2.`$rowtime` - INTERVAL '1' MINUTE\n" + + " AND e2.`$rowtime` + INTERVAL '1' MINUTE"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseExecPlan() { util.verifyExecPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @Test void testThreeWayLeftOuterJoinWithWhereClauseExplain() { util.verifyExplain( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON u.user_id_0 = p.user_id_2 " + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + "WHERE u.name = 'Gus' AND p.price > 10"); } @@ -332,52 +534,1095 @@ void testThreeWayLeftOuterJoinWithWhereClauseExplain() { void testRegularJoinsAreMergedApartFromTemporalJoin() { // Regular joins should still be eligible for MultiJoin but not mixed with temporal joins util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, temporal.age " - + "FROM Users u " - + "INNER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "INNER JOIN (" - + " SELECT s.user_id, l.age " - + " FROM StreamTable s " - + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l " - + " ON s.user_id = l.id" - + ") temporal ON u.user_id_0 = temporal.user_id"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " temporal.age " + + "FROM Users u\n" + + "INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "INNER JOIN (\n" + + " SELECT s.user_id, l.age\n" + + " FROM StreamTable s\n" + + " JOIN LookupTable FOR SYSTEM_TIME AS OF s.proctime AS l\n" + + " ON s.user_id = l.id\n" + + ") temporal ON u.user_id = temporal.user_id"); } @Test void testFourWayJoinTransitiveCommonJoinKeyRelPlan() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id, s.location " - + "FROM Users u " - + "LEFT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "LEFT JOIN Payments p ON o.user_id_1 = p.user_id_2 " - + "LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON o.user_id = p.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); } /* Update this to supported with FLINK-37973 https://issues.apache.org/jira/browse/FLINK-37973 */ @Test void testRightJoinNotSupported() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "RIGHT JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "RIGHT JOIN Payments p ON o.user_id_1 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "RIGHT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "RIGHT JOIN Payments p\n" + + " ON o.user_id = p.user_id"); } @Test void testFullOuterNotSupported() { util.verifyRelPlan( - "SELECT u.user_id_0, u.name, o.order_id, p.payment_id " - + "FROM Users u " - + "FULL OUTER JOIN Orders o ON u.user_id_0 = o.user_id_1 " - + "FULL OUTER JOIN Payments p ON o.user_id_1 = p.user_id_2"); + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id\n" + + "FROM Users u\n" + + "FULL OUTER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "FULL OUTER JOIN Payments p\n" + + " ON o.user_id = p.user_id"); } @Test void testThreeWayJoinWithTimeAttributesMaterialization() { util.verifyRelPlan( - "SELECT u.name, u.proctime, o.rowtime, p.price " - + "FROM UsersWithProctime u " - + "JOIN OrdersWithRowtime o ON u.user_id_0 = o.user_id_1 " - + "JOIN Payments p ON u.user_id_0 = p.user_id_2"); + "\nSELECT\n" + + " u.name,\n" + + " u.proctime,\n" + + " o.`$rowtime`,\n" + + " p.price\n" + + "FROM UsersWithProctime u\n" + + "JOIN OrdersWithRowtime o\n" + + " ON u.user_id = o.user_id\n" + + "JOIN Payments p\n" + + " ON u.user_id = p.user_id"); + } + + @Test + void testPreservesUpsertKeyTwoWayLeftJoinOrders() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM OrdersPK o\n" + + "LEFT JOIN UsersPK u\n" + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyTwoWayInnerJoinOrders() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM UsersPK u\n" + + "INNER JOIN OrdersPK o\n" + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyTwoWayInnerJoinOrdersDoesNot() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrdersSimplePK (" + + " order_id STRING NOT NULL," + + " user_id STRING NOT NULL," + + " product STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (order_id) NOT ENFORCED" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE sink_two_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " product STRING," + + " user_region_id INT," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`order_id`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "\nINSERT INTO sink_two_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " o.product,\n" + + " u.region_id\n" + + "FROM UsersPK u\n" + + "INNER JOIN OrdersSimplePK o\n" + + " ON u.user_id = o.user_id"); + } + + @Test + void testPreservesUpsertKeyThreeWayJoin() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_three_way (" + + " `user_id` STRING NOT NULL," + + " `order_id` STRING NOT NULL," + + " `user_id2` STRING NOT NULL," + + " `payment_id` STRING NOT NULL," + + " `user_id3` STRING NOT NULL," + + " `description` STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id2`, `payment_id`, `user_id3`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "\nINSERT INTO sink_three_way\n" + + "SELECT\n" + + " o.user_id,\n" + + " o.order_id,\n" + + " p.user_id,\n" + + " p.payment_id,\n" + + " u.user_id,\n" + + " u.description\n" + + "FROM UsersPK u\n" + + "JOIN OrdersPK o\n" + + " ON o.user_id = u.user_id\n" + + "JOIN PaymentsPK p\n" + + " ON o.user_id = p.user_id"); + } + + @Test + void testPreservesUpsertKeyFourWayComplex() { + util.tableEnv() + .executeSql( + "CREATE TABLE sink_four_way (" + + " user_id STRING NOT NULL," + + " order_id STRING NOT NULL," + + " user_id1 STRING NOT NULL," + + " payment_id STRING NOT NULL," + + " user_id2 STRING NOT NULL," + + " name STRING," + + " location STRING," + + " CONSTRAINT `PRIMARY` PRIMARY KEY (`user_id`, `order_id`, `user_id1`, `payment_id`, `user_id2`) NOT ENFORCED" + + ") WITH (" + + " 'connector' = 'values'," + + " 'sink-insert-only' = 'false'" + + ")"); + + util.verifyRelPlanInsert( + "\nINSERT INTO sink_four_way\n" + + "SELECT\n" + + " u.user_id,\n" + + " o.order_id,\n" + + " o.user_id,\n" + + " p.payment_id,\n" + + " p.user_id,\n" + + " u.name,\n" + + " a.location\n" + + "FROM UsersPK u\n" + + "JOIN OrdersPK o\n" + + " ON u.user_id = o.user_id\n" + + " AND o.product IS NOT NULL\n" + + "JOIN PaymentsPK p\n" + + " ON u.user_id = p.user_id\n" + + " AND p.price >= 0\n" + + "JOIN AddressPK a\n" + + " ON u.user_id = a.user_id\n" + + " AND a.location IS NOT NULL"); + } + + @Test + void testMultiSinkOnMultiJoinedView() { + util.tableEnv() + .executeSql( + "create temporary table src1 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + util.tableEnv() + .executeSql( + "create temporary table src2 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + util.tableEnv() + .executeSql( + "create temporary table sink1 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d bigint\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + util.tableEnv() + .executeSql( + "create temporary table sink2 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d string\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + util.tableEnv() + .executeSql( + "create temporary view v1 as\n" + + "select\n" + + " t1.a as a, t1.`day` as `day`, t2.b as b, t2.c as c\n" + + "from (\n" + + " select a, b, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') as `day`\n" + + " from src1\n" + + " ) t1\n" + + "join (\n" + + " select b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `day`, c, d\n" + + " from src2\n" + + ") t2\n" + + " on t1.a = t2.d"); + + StatementSet stmtSet = util.tableEnv().createStatementSet(); + stmtSet.addInsertSql( + "insert into sink1\n" + + " select a, `day`, sum(b), count(distinct c)\n" + + " from v1\n" + + " group by a, `day`"); + stmtSet.addInsertSql( + "insert into sink2\n" + + " select a, `day`, b, c\n" + + " from v1\n" + + " where b > 100"); + + util.doVerifyPlan( + stmtSet, + new ExplainDetail[] {ExplainDetail.PLAN_ADVICE}, + false, + new Enumeration.Value[] {PlanKind.OPT_REL_WITH_ADVICE()}, + () -> UNIT, + false, + false); + } + + /* + * Calcite adds a LogicalProject to compute expressions such as UPPER and FLOOR + * on the necessary fields. As a result, the planner cannot fuse all joins into + * a single MultiJoin node initially. + */ + @Test + @Tag("multijoin-chain-expected") + void testFourWayJoinWithFunctionInConditionMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + " AND UPPER(u.name) = UPPER(p.payment_id)\n" + + " AND (FLOOR(u.cash) >= FLOOR(p.price) OR p.price < 0)\n" + + "LEFT JOIN Shipments s\n" + + " ON p.payment_id = s.location"); + } + + /* + * We expect the join inputs to **not** merge into a single MultiJoin node in this case, + * because `documents.common_id` is different from `other_documents.common_id`. + */ + @Test + @Tag("no-common-join-key") + void testComplexCommonJoinKeyMissingProjectionNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE Assignments (" + + " assignment_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " detail_id STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Documents (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " creator_nm STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Assignments assignments\n" + + "LEFT JOIN Documents AS documents\n" + + " ON assignments.detail_id = documents.detail_id\n" + + " AND assignments.common_id = documents.common_id\n" + + "LEFT JOIN Documents AS other_documents\n" + + " ON assignments.user_id = other_documents.common_id"); + } + + @Test + void testComplexCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE Assignments (" + + " assignment_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " detail_id STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Customers (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " name STRING," + + " depart_num STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Documents (" + + " detail_id STRING PRIMARY KEY NOT ENFORCED," + + " creator_nm STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE PhaseDetails (" + + " phase_id STRING PRIMARY KEY NOT ENFORCED," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Organizations (" + + " org_id STRING PRIMARY KEY NOT ENFORCED," + + " org_name STRING," + + " common_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyExecPlan( + "\nSELECT *\n" + + "FROM Assignments assignments\n" + + "LEFT JOIN Customers AS customer\n" + + " ON assignments.user_id = customer.user_id\n" + + " AND assignments.common_id = customer.common_id\n" + + "LEFT JOIN Documents AS documents\n" + + " ON assignments.detail_id = documents.detail_id\n" + + " AND assignments.common_id = documents.common_id\n" + + "LEFT JOIN PhaseDetails AS phase_details\n" + + " ON documents.common_id = phase_details.common_id\n" + + "LEFT JOIN Organizations AS organizations\n" + + " ON customer.depart_num = organizations.org_id\n" + + " AND customer.common_id = organizations.common_id\n" + + "LEFT JOIN Customers AS creators\n" + + " ON documents.creator_nm = creators.depart_num\n" + + " AND documents.common_id = creators.common_id"); + } + + @Test + @Tag("no-common-join-key") + void testComplexConditionalLogicWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductCategories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " is_premium BOOLEAN," + + " discount_rate DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE ProductReviews (" + + " review_id STRING PRIMARY KEY NOT ENFORCED," + + " product_id STRING," + + " rating INT," + + " is_verified BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " pc.category_name,\n" + + " CASE\n" + + " WHEN pc.is_premium = true AND p.price > 1000 THEN 'High-Value Premium'\n" + + " WHEN pc.is_premium = true THEN 'Premium'\n" + + " WHEN p.price > 500 THEN 'Standard High-Value'\n" + + " ELSE 'Standard'\n" + + " END AS product_tier,\n" + + " CASE\n" + + " WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended'\n" + + " WHEN pr.rating >= 3 THEN 'Recommended'\n" + + " WHEN pr.rating >= 2 THEN 'Average'\n" + + " ELSE 'Not Recommended'\n" + + " END AS recommendation_status,\n" + + " CASE\n" + + " WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate)\n" + + " ELSE p.price\n" + + " END AS final_price\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN ProductCategories pc\n" + + " ON o.product = pc.category_id\n" + + "LEFT JOIN ProductReviews pr\n" + + " ON o.product = pr.product_id"); + } + + @Test + @Tag("no-common-join-key") + void testComplexCTEWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderStatus (" + + " status_id STRING PRIMARY KEY NOT ENFORCED," + + " status_name STRING," + + " is_final BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE PaymentMethods (" + + " method_id STRING PRIMARY KEY NOT ENFORCED," + + " method_name STRING," + + " processing_fee DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH user_orders AS (\n" + + " SELECT u.user_id, u.name, o.order_id, o.product, p.payment_id, p.price\n" + + " FROM Users u\n" + + " LEFT JOIN Orders o ON\n" + + " u.user_id = o.user_id\n" + + " LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "),\n" + + "order_details AS (\n" + + " SELECT uo.*, os.status_name, os.is_final, pm.method_name, pm.processing_fee\n" + + " FROM user_orders uo\n" + + " LEFT JOIN OrderStatus os\n" + + " ON uo.order_id = os.status_id\n" + + " LEFT JOIN PaymentMethods pm\n" + + " ON uo.payment_id = pm.method_id\n" + + "),\n" + + "final_summary AS (\n" + + " SELECT\n" + + " user_id,\n" + + " name,\n" + + " COUNT(order_id) as total_orders,\n" + + " SUM(price) as total_spent,\n" + + " AVG(price) as avg_order_value,\n" + + " COUNT(CASE WHEN is_final = true THEN 1 END) as completed_orders\n" + + " FROM order_details\n" + + " GROUP BY user_id, name\n" + + ")\n" + + "SELECT * FROM final_summary"); + } + + @Test + @Tag("no-common-join-key") + void testAggregationAndGroupingWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderItems (" + + " item_id STRING PRIMARY KEY NOT ENFORCED," + + " order_id STRING," + + " product_name STRING," + + " quantity INT," + + " unit_price DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE ProductCategories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " parent_category STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " pc.category_name,\n" + + " COUNT(DISTINCT o.order_id) as order_count,\n" + + " SUM(oi.quantity) as total_items,\n" + + " SUM(oi.quantity * oi.unit_price) as total_value,\n" + + " AVG(oi.unit_price) as avg_item_price,\n" + + " MAX(p.price) as max_payment,\n" + + " MIN(p.price) as min_payment,\n" + + " COUNT(CASE WHEN oi.quantity > 5 THEN 1 END) as bulk_orders\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN OrderItems oi\n" + + " ON o.order_id = oi.order_id\n" + + "LEFT JOIN ProductCategories pc\n" + + " ON oi.product_name = pc.category_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "GROUP BY u.user_id, u.name, pc.category_name\n" + + "HAVING COUNT(DISTINCT o.order_id) > 0"); + } + + @Test + @Tag("no-common-join-key") + void testFunctionAndExpressionWithMultiJoinNoCommonJoinKey() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductDetails (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " product_name STRING," + + " description STRING," + + " created_date BIGINT," + + " tags STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE UserPreferences (" + + " user_id STRING PRIMARY KEY NOT ENFORCED," + + " preferred_category STRING," + + " notification_level STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " UPPER(u.name) as user_name_upper,\n" + + " LOWER(o.product) as product_lower,\n" + + " CONCAT(u.name, ' - ', o.product) as user_product,\n" + + " SUBSTRING(pd.description, 1, 50) as description_preview,\n" + + " CHAR_LENGTH(pd.description) as description_length,\n" + + " FLOOR(p.price / 100.0) * 100 as price_rounded,\n" + + " CASE\n" + + " WHEN p.price > 1000 THEN 'High'\n" + + " WHEN p.price > 500 THEN 'Medium'\n" + + " ELSE 'Low'\n" + + " END as price_tier,\n" + + " REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags,\n" + + " TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created,\n" + + " COALESCE(up.preferred_category, 'None') as user_preference,\n" + + " CASE\n" + + " WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates'\n" + + " WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates'\n" + + " ELSE 'Weekly Updates'\n" + + " END as notification_frequency\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN ProductDetails pd\n" + + " ON o.product = pd.product_id\n" + + "LEFT JOIN UserPreferences up\n" + + " ON u.user_id = up.user_id"); + } + + /* + * Calcite automatically generates LogicalProject nodes for nested field access. + * As a result, each join input in this test is wrapped in a projection, which prevents + * the planner from fusing all joins into a single MultiJoin node initially. + * Therefore, in this test, each Join is still converted to a MultiJoin individually. + */ + @Test + @Tag("multijoin-chain-expected") + void testJoinConditionHasNestedFieldsMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE Developers (" + + " developer_id STRING PRIMARY KEY NOT ENFORCED," + + " person ROW>," + + " experience_years INT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE SupportTickets (" + + " ticket_id STRING PRIMARY KEY NOT ENFORCED," + + " reporter ROW>," + + " issue STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Feedback (" + + " feedback_id STRING PRIMARY KEY NOT ENFORCED," + + " author ROW>," + + " message STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Subscriptions (" + + " sub_id STRING PRIMARY KEY NOT ENFORCED," + + " subscriber ROW>," + + " active BOOLEAN" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " d.developer_id,\n" + + " d.person.info.name AS developer_name,\n" + + " s.ticket_id,\n" + + " s.reporter.info.priority AS ticket_priority,\n" + + " f.feedback_id,\n" + + " f.author.info.rating AS feedback_rating,\n" + + " sub.sub_id,\n" + + " sub.subscriber.info.plan AS subscription_plan\n" + + "FROM Developers AS d\n" + + "LEFT JOIN SupportTickets AS s\n" + + " ON d.person.info.id = s.reporter.info.id\n" + + "LEFT JOIN Feedback AS f\n" + + " ON d.person.info.id = f.author.info.id\n" + + "LEFT JOIN Subscriptions AS sub\n" + + " ON d.person.info.id = sub.subscriber.info.id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testComplexNestedCTEWithAggregationAndFunctionsMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE OrderMetrics (" + + " metric_id STRING PRIMARY KEY NOT ENFORCED," + + " order_id STRING," + + " metric_type STRING," + + " metric_value DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH base_orders AS (\n" + + " SELECT u.user_id, u.name, o.order_id, p.payment_id, p.price\n" + + " FROM Users u\n" + + " INNER JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + " INNER JOIN Payments p\n" + + " ON u.user_id = p.user_id\n" + + "),\n" + + "enriched_orders AS (\n" + + " SELECT\n" + + " bo.*,\n" + + " om.metric_type,\n" + + " om.metric_value,\n" + + " CASE\n" + + " WHEN bo.price > 1000 THEN 'Premium'\n" + + " WHEN bo.price > 500 THEN 'Standard'\n" + + " ELSE 'Basic'\n" + + " END as order_tier\n" + + " FROM base_orders bo\n" + + " LEFT JOIN OrderMetrics om\n" + + " ON bo.order_id = om.order_id\n" + + "),\n" + + "aggregated_metrics AS (\n" + + " SELECT\n" + + " user_id,\n" + + " name,\n" + + " COUNT(DISTINCT order_id) as total_orders,\n" + + " SUM(price) as total_spent,\n" + + " AVG(price) as avg_order_value,\n" + + " MAX(metric_value) as max_metric,\n" + + " MIN(metric_value) as min_metric,\n" + + " COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders\n" + + " FROM enriched_orders\n" + + " GROUP BY user_id, name\n" + + ")\n" + + "SELECT\n" + + " user_id,\n" + + " UPPER(name) as user_name,\n" + + " total_orders,\n" + + " ROUND(total_spent, 2) as total_spent_rounded,\n" + + " ROUND(avg_order_value, 2) as avg_order_value_rounded,\n" + + " CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary,\n" + + " CASE\n" + + " WHEN total_orders > 10 THEN 'Frequent Customer'\n" + + " WHEN total_orders > 5 THEN 'Regular Customer'\n" + + " ELSE 'Occasional Customer'\n" + + " END as customer_type\n" + + "FROM aggregated_metrics\n" + + "WHERE total_spent > 0"); + } + + @Test + void testJoinOfProjections() { + util.verifyRelPlan( + "\nSELECT u.user_id, o.order_id, o.product, p.price, s.location\n" + + "FROM (SELECT user_id, name, cash FROM Users WHERE cash > 100) AS u\n" + + "JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p\n" + + " ON u.user_id = p.user_id\n" + + "LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testJoinWithNestedSubqueryMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Users u\n" + + "JOIN (\n" + + " SELECT o.user_id, o.order_id, p.payment_id, p.price\n" + + " FROM Orders o\n" + + " JOIN (\n" + + " SELECT payment_id, user_id, price\n" + + " FROM Payments\n" + + " WHERE price > 100\n" + + " ) AS p\n" + + " ON o.user_id = p.user_id\n" + + ") AS op\n" + + "ON u.user_id = op.user_id"); + } + + @Test + void testCTEWithMultiJoinV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE Departments (" + + " dept_id STRING PRIMARY KEY NOT ENFORCED," + + " dept_name STRING," + + " budget DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Projects (" + + " project_id STRING PRIMARY KEY NOT ENFORCED," + + " project_name STRING," + + " dept_id STRING," + + " status STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nWITH high_budget_depts AS (\n" + + " SELECT dept_id, dept_name, budget\n" + + " FROM Departments\n" + + " WHERE budget > 600000\n" + + "),\n" + + "active_projects AS (\n" + + " SELECT project_id, project_name, dept_id\n" + + " FROM Projects\n" + + " WHERE status = 'ACTIVE'\n" + + ")\n" + + "SELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " hbd.dept_name,\n" + + " ap.project_name,\n" + + " hbd.budget\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN high_budget_depts hbd\n" + + " ON o.user_id = hbd.dept_id\n" + + "LEFT JOIN active_projects ap\n" + + " ON hbd.dept_id = ap.dept_id"); + } + + @Test + void testWithOrInJoinCondition() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON u.user_id = p.user_id OR u.name = p.payment_id\n" + + "LEFT JOIN Shipments s\n" + + " ON p.user_id = s.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testWithCastCommonJoinKeyToIntegerMultiJoinChainExpected() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON CAST(u.user_id as INTEGER) = CAST(p.user_id as INTEGER)\n" + + "LEFT JOIN Shipments s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + void testWithCastCommonJoinKeyToVarchar() { + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " p.payment_id,\n" + + " s.location\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON o.user_id = u.user_id\n" + + "LEFT JOIN Payments p\n" + + " ON CAST(u.user_id as VARCHAR) = CAST(p.user_id as VARCHAR)\n" + + "LEFT JOIN Shipments s\n" + + " ON u.user_id = s.user_id"); + } + + @Test + void testAggregationAndGroupingWithMultiJoinV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE Categories (" + + " category_id STRING PRIMARY KEY NOT ENFORCED," + + " category_name STRING," + + " parent_category STRING," + + " user_id STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Sales (" + + " sale_id STRING PRIMARY KEY NOT ENFORCED," + + " user_id STRING," + + " product_id STRING," + + " amount DOUBLE," + + " sale_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " c.category_name,\n" + + " COUNT(DISTINCT u.user_id) AS unique_users,\n" + + " COUNT(s.sale_id) AS total_sales,\n" + + " SUM(s.amount) AS total_revenue,\n" + + " AVG(s.amount) AS avg_sale_amount,\n" + + " MAX(s.amount) AS max_sale_amount\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN Categories c\n" + + " ON u.user_id = c.user_id AND o.product = c.category_id\n" + + "LEFT JOIN Sales s\n" + + " ON u.user_id = s.user_id\n" + + "GROUP BY c.category_name\n" + + "HAVING COUNT(s.sale_id) > 0"); + } + + @Test + void testSameTableMultipleAliases() { + util.verifyRelPlan( + "\nSELECT *\n" + + "FROM Users u\n" + + "LEFT JOIN Users u1\n" + + " ON u.user_id = u1.user_id\n" + + "LEFT JOIN Users u2\n" + + " ON u1.user_id = u2.user_id\n" + + "LEFT JOIN Users u3\n" + + " ON u2.user_id = u3.user_id"); + } + + @Test + @Tag("multijoin-chain-expected") + void testWithExpressionInJoinConditionMultiJoinChainExpected() { + util.tableEnv() + .executeSql( + "CREATE TABLE Products (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " price DOUBLE," + + " discount DOUBLE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Sales (" + + " sale_id STRING PRIMARY KEY NOT ENFORCED," + + " product_key DOUBLE," + + " quantity INT" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Promotions (" + + " promo_id STRING PRIMARY KEY NOT ENFORCED," + + " product_key DOUBLE," + + " promo_text STRING" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " p.product_id,\n" + + " (p.price - p.discount) AS net_price,\n" + + " s.quantity,\n" + + " pr.promo_text\n" + + "FROM Products AS p\n" + + "LEFT JOIN Sales AS s\n" + + " ON (p.price - p.discount) = s.product_key\n" + + "LEFT JOIN Promotions AS pr\n" + + " ON (p.price - p.discount) = pr.product_key\n" + + "WHERE (p.price - p.discount) > 100"); + } + + @Test + @Tag("no-common-join-key") + void testFunctionAndExpressionWithMultiJoinNoCommonJoinKeyV2() { + util.tableEnv() + .executeSql( + "CREATE TABLE ProductDetails (" + + " product_id STRING PRIMARY KEY NOT ENFORCED," + + " product_name STRING," + + " price DOUBLE," + + " weight DOUBLE," + + " created_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.tableEnv() + .executeSql( + "CREATE TABLE Reviews (" + + " review_id STRING PRIMARY KEY NOT ENFORCED," + + " product_id STRING," + + " rating INT," + + " review_text STRING," + + " review_date DATE" + + ") WITH ('connector' = 'values', 'changelog-mode' = 'I,UA,D')"); + + util.verifyRelPlan( + "\nSELECT\n" + + " u.user_id,\n" + + " u.name,\n" + + " o.order_id,\n" + + " pd.product_name,\n" + + " pd.price,\n" + + " ROUND(pd.price * 1.1, 2) AS price_with_tax,\n" + + " CONCAT('Product: ', pd.product_name) AS product_description,\n" + + " CHAR_LENGTH(r.review_text) AS review_length,\n" + + " UPPER(SUBSTRING(r.review_text, 1, 10)) AS review_preview,\n" + + " CASE\n" + + " WHEN r.rating >= 4 THEN 'High Rating'\n" + + " WHEN r.rating >= 3 THEN 'Medium Rating'\n" + + " ELSE 'Low Rating'\n" + + " END AS rating_category,\n" + + " TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created\n" + + "FROM Users u\n" + + "LEFT JOIN Orders o\n" + + " ON u.user_id = o.user_id\n" + + "LEFT JOIN ProductDetails pd\n" + + " ON o.product = pd.product_id\n" + + "LEFT JOIN Reviews r\n" + + " ON pd.product_id = r.product_id"); + } + + @Test + void testCrossJoinUnnestWithMultiJoinInsert() { + util.tableEnv() + .executeSql( + "CREATE TABLE UnnestSink (" + + " detail_id STRING," + + " element_data STRING," + + " data_value_id INT," + + " user_id STRING," + + " order_id STRING" + + ") WITH ('connector' = 'values', 'sink-insert-only' = 'false')"); + + util.verifyRelPlanInsert( + "\nINSERT INTO UnnestSink\n" + + "(\n" + + " detail_id,\n" + + " element_data,\n" + + " data_value_id,\n" + + " user_id,\n" + + " order_id\n" + + ")\n" + + "SELECT\n" + + " d.detail_id,\n" + + " TRIM(REGEXP_REPLACE(edata, '[\\[\\]\\\"]', '')) AS element_data,\n" + + " ARRAY_POSITION(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"'), edata) as data_value_id,\n" + + " d.user_id,\n" + + " o.order_id\n" + + "FROM Detail d\n" + + "INNER JOIN Orders o\n" + + " ON o.user_id = d.user_id\n" + + "INNER JOIN Payments p\n" + + " ON p.user_id = d.user_id\n" + + "LEFT JOIN Shipments s\n" + + " ON s.user_id = d.user_id\n" + + "CROSS JOIN UNNEST(split(REGEXP_REPLACE(d.data, '^\\[\"|\"\\]$', '') , '\", \"')) AS T(edata)\n" + + "WHERE NOT (s.location IS NOT NULL)"); } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 7df8d36563af3..6c3a455d789a5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -16,13 +16,648 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + 5 THEN 1 END) as bulk_orders +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN OrderItems oi + ON o.order_id = oi.order_id +LEFT JOIN ProductCategories pc + ON oi.product_name = pc.category_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +GROUP BY u.user_id, u.name, pc.category_name +HAVING COUNT(DISTINCT o.order_id) > 0]]> + + + ($3, 0)]) ++- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)]) + +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)]) + +- LogicalJoin(condition=[=($0, $16)], joinType=[left]) + :- LogicalJoin(condition=[=($8, $11)], joinType=[left]) + : :- LogicalJoin(condition=[=($3, $7)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + (order_count, 0)]) ++- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) + +- Exchange(distribution=[hash[user_id, name, category_name]]) + +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name]) + : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)]) + : :- Exchange(distribution=[hash[product_name]]) + : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price]) + : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : : :- Exchange(distribution=[hash[order_id]]) + : : : +- Calc(select=[user_id, name, order_id]) + : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) + : : : :- Exchange(distribution=[hash[user_id]]) + : : : : +- ChangelogNormalize(key=[user_id]) + : : : : +- Exchange(distribution=[hash[user_id]]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : : +- Exchange(distribution=[hash[order_id]]) + : : +- Calc(select=[order_id, product_name, quantity, unit_price]) + : : +- ChangelogNormalize(key=[item_id]) + : : +- Exchange(distribution=[hash[item_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + : +- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) +]]> + + + + + 0]]> + + + ($2, 0)]) ++- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)]) + +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13]) + +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) + :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) +]]> + + + (total_sales, 0)]) ++- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) + +- Exchange(distribution=[hash[category_name]]) + +- Calc(select=[category_name, user_id, sale_id, amount]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[sale_id]) + +- Exchange(distribution=[hash[sale_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount]) +]]> + + + + + 600000 +), +active_projects AS ( + SELECT project_id, project_name, dept_id + FROM Projects + WHERE status = 'ACTIVE' +) +SELECT + u.user_id, + u.name, + o.order_id, + hbd.dept_name, + ap.project_name, + hbd.budget +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN high_budget_depts hbd + ON o.user_id = hbd.dept_id +LEFT JOIN active_projects ap + ON hbd.dept_id = ap.dept_id]]> + + + ($2, 600000)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]]) + +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2]) + +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')]) + +- LogicalTableScan(table=[[default_catalog, default_database, Projects]]) +]]> + + + (budget, 600000)]) + : +- Exchange(distribution=[hash[dept_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) + +- Exchange(distribution=[hash[dept_id]]) + +- Calc(select=[project_name, dept_id]) + +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) + +- Exchange(distribution=[hash[project_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1000 THEN 'High-Value Premium' + WHEN pc.is_premium = true THEN 'Premium' + WHEN p.price > 500 THEN 'Standard High-Value' + ELSE 'Standard' + END AS product_tier, + CASE + WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended' + WHEN pr.rating >= 3 THEN 'Recommended' + WHEN pr.rating >= 2 THEN 'Average' + ELSE 'Not Recommended' + END AS recommendation_status, + CASE + WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate) + ELSE p.price + END AS final_price +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductCategories pc + ON o.product = pc.category_id +LEFT JOIN ProductReviews pr + ON o.product = pr.product_id]]> + + + ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)]) ++- LogicalJoin(condition=[=($5, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) + +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]]) +]]> + + + (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) + :- Exchange(distribution=[hash[product]]) + : +- Calc(select=[user_id, order_id, product, payment_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + :- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) + +- Exchange(distribution=[hash[product_id]]) + +- Calc(select=[product_id, rating, is_verified]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) +]]> + + + + + 1000 THEN 'Premium' + WHEN bo.price > 500 THEN 'Standard' + ELSE 'Basic' + END as order_tier + FROM base_orders bo + LEFT JOIN OrderMetrics om + ON bo.order_id = om.order_id +), +aggregated_metrics AS ( + SELECT + user_id, + name, + COUNT(DISTINCT order_id) as total_orders, + SUM(price) as total_spent, + AVG(price) as avg_order_value, + MAX(metric_value) as max_metric, + MIN(metric_value) as min_metric, + COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders + FROM enriched_orders + GROUP BY user_id, name +) +SELECT + user_id, + UPPER(name) as user_name, + total_orders, + ROUND(total_spent, 2) as total_spent_rounded, + ROUND(avg_order_value, 2) as avg_order_value_rounded, + CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary, + CASE + WHEN total_orders > 10 THEN 'Frequent Customer' + WHEN total_orders > 5 THEN 'Regular Customer' + ELSE 'Occasional Customer' + END as customer_type +FROM aggregated_metrics +WHERE total_spent > 0]]> + + + ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")]) ++- LogicalFilter(condition=[>($3, 0)]) + +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)]) + +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")]) + +- LogicalJoin(condition=[=($2, $6)], joinType=[left]) + :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7]) + : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]]) +]]> + + + (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) ++- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) + +- Exchange(distribution=[hash[user_id, name]]) + +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)]) + :- Exchange(distribution=[hash[order_id]]) + : +- Calc(select=[user_id, name, order_id, price]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- Exchange(distribution=[hash[order_id]]) + +- Calc(select=[order_id, metric_value]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id]) +]]> + + + + + + + + + + + + + - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -34,25 +669,25 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> =($2, $7), <($7, 0)))], joinType=[inner]) : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -62,44 +697,264 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6], locati +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) == Optimized Physical Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) == Optimized Execution Plan == -Calc(select=[user_id_0, name, order_id, payment_id, location]) -+- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - + - + = p.price OR p.price < 0) +LEFT JOIN Shipments s + ON p.user_id = s.user_id]]> =($2, $7), <($7, 0)))], joinType=[inner]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + + + =(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[user_id]]) + : +- ChangelogNormalize(key=[user_id]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + = FLOOR(p.price) OR p.price < 0) +LEFT JOIN Shipments s + ON p.payment_id = s.location]]> + + + =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left]) + : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)]) + : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + + + =(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) +: :- Exchange(distribution=[hash[user_id, $f6]]) +: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6]) +: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) +: : :- Exchange(distribution=[hash[user_id]]) +: : : +- ChangelogNormalize(key=[user_id]) +: : : +- Exchange(distribution=[hash[user_id]]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) +: : +- Exchange(distribution=[hash[user_id]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) +: +- Exchange(distribution=[hash[user_id, $f3]]) +: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) +: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ++- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location]) +]]> + + + + + + + + + + + + + 1000 THEN 'High' + WHEN p.price > 500 THEN 'Medium' + ELSE 'Low' + END as price_tier, + REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags, + TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created, + COALESCE(up.preferred_category, 'None') as user_preference, + CASE + WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates' + WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates' + ELSE 'Weekly Updates' + END as notification_frequency +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN Payments p + ON u.user_id = p.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN UserPreferences up + ON u.user_id = up.user_id]]> + + + ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")]) ++- LogicalJoin(condition=[=($0, $14)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]]) +]]> + + + (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, name, product, price, description, created_date, tags]) + : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) + : :- Exchange(distribution=[hash[product]]) + : : +- Calc(select=[user_id, name, product, price]) + : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, UserPreferences]], fields=[user_id, preferred_category, notification_level]) +]]> + + + + + = 4 THEN 'High Rating' + WHEN r.rating >= 3 THEN 'Medium Rating' + ELSE 'Low Rating' + END AS rating_category, + TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created +FROM Users u +LEFT JOIN Orders o + ON u.user_id = o.user_id +LEFT JOIN ProductDetails pd + ON o.product = pd.product_id +LEFT JOIN Reviews r + ON pd.product_id = r.product_id]]> + + + =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER]) ++- LogicalJoin(condition=[=($6, $12)], joinType=[left]) + :- LogicalJoin(condition=[=($5, $6)], joinType=[left]) + : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]]) +]]> + + + =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) ++- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)]) + :- Exchange(distribution=[hash[product]]) + : +- Calc(select=[user_id, name, order_id, product]) + : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date]) + +- Exchange(distribution=[hash[product_id]]) + +- Calc(select=[product_id, rating, review_text]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id]) +]]> + + + + + + + + =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) + :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) + +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) +]]> + + + =($rowtime, -($rowtime0, 60000:INTERVAL MINUTE)), <=($rowtime, +($rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, $rowtime, id0, price, $rowtime0]) + :- Exchange(distribution=[hash[id]]) + : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) + +- Exchange(distribution=[hash[id]]) + +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) +]]> + + + + + + + + + + + + + + + + 100) AS u +JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o + ON u.user_id = o.user_id +LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p + ON u.user_id = p.user_id +LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s + ON u.user_id = s.user_id]]> + + + ($2, 100)]) + : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2]) + : : +- LogicalFilter(condition=[IS NOT NULL($2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + : +- LogicalProject(user_id=[$2], price=[$1]) + : +- LogicalFilter(condition=[>($1, 50)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) + +- LogicalProject(user_id=[$1], location=[$0]) + +- LogicalFilter(condition=[IS NOT NULL($0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +]]> + + + (cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id, price], where=[>(price, 50)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[], project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[user_id, location], where=[IS NOT NULL(location)]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments, filter=[]]], fields=[location, user_id]) +]]> + + + + + 100 + ) AS p + ON o.user_id = p.user_id +) AS op +ON u.user_id = op.user_id]]> + + + ($1, 100)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + (price, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +]]> + + + + + (b, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). +advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. + +related rel plan: +Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) ++- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA,D]) + + +]]> + + + + + = 0 +JOIN AddressPK a + ON u.user_id = a.user_id + AND a.location IS NOT NULL]]> + + + =($9, 0))], joinType=[inner]) + : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) + : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) + +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) +]]> + + + =(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + +- Exchange(distribution=[hash[user_id]]) + +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, AddressPK, filter=[]]], fields=[user_id, location]) ]]> - + - = p.price OR p.price < 0) LEFT JOIN Shipments s ON p.user_id_2 = s.user_id_3]]> + =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) +LogicalSink(table=[default_catalog.default_database.sink_three_way], fields=[user_id, order_id, user_id0, payment_id, user_id1, description]) ++- LogicalProject(user_id=[$5], order_id=[$4], user_id0=[$8], payment_id=[$7], user_id1=[$0], description=[$3]) + +- LogicalJoin(condition=[=($5, $8)], joinType=[inner]) + :- LogicalJoin(condition=[=($5, $0)], joinType=[inner]) + : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) + +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) ]]> =($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - :- Exchange(distribution=[hash[user_id_2]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2]) - +- Exchange(distribution=[hash[user_id_3]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3]) +Sink(table=[default_catalog.default_database.sink_three_way], fields=[user_id, order_id, user_id0, payment_id, user_id1, description]) ++- Calc(select=[user_id0 AS user_id, order_id, user_id1 AS user_id0, payment_id, user_id AS user_id1, description]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id, user_id), (payment_id, user_id)], joinConditions=[=(user_id0, user_id), =(user_id0, user_id1)], select=[user_id,description,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) description, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, UsersPK, project=[user_id, description], metadata=[]]], fields=[user_id, description]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, OrdersPK, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) ]]> - + - + - + - + - + - + =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) - :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) - +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) +LogicalSink(table=[default_catalog.default_database.sink_two_way], fields=[user_id, order_id, product, region_id]) ++- LogicalProject(user_id=[$1], order_id=[$0], product=[$2], region_id=[$5]) + +- LogicalJoin(condition=[=($3, $1)], joinType=[left]) + :- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) + +- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) ]]> =(rowtime, -(rowtime0, 60000:INTERVAL MINUTE)), <=(rowtime, +(rowtime0, 60000:INTERVAL MINUTE)))], select=[id, val, rowtime, id0, price, rowtime0]) - :- Exchange(distribution=[hash[id]]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, rowtime]) - +- Exchange(distribution=[hash[id]]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 5000:INTERVAL SECOND)]) - +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, rowtime]) +Sink(table=[default_catalog.default_database.sink_two_way], fields=[user_id, order_id, product, region_id]) ++- Calc(select=[user_id, order_id, product, region_id]) + +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(order_id, user_id), (user_id)], joinConditions=[=(user_id0, user_id)], select=[order_id,user_id,product,user_id0,region_id], rowType=[RecordType(VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id, VARCHAR(2147483647) product, VARCHAR(2147483647) user_id0, INTEGER region_id)]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, OrdersPK]], fields=[order_id, user_id, product]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, UsersPK, project=[user_id, region_id], metadata=[]]], fields=[user_id, region_id]) ]]> - + - + + + + + + + + + + + + - + - + - + - + - - - + + + + + + + + - + - + + + + + + + + + + + + - + - + - + - + - 10]]> + ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) ++- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) ]]> - + (price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, name, order_id, payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) ]]> - - - - - - + + - - - - + + + + + + 10]]> + + + ($7, 10))]) + +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) + :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) + : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) +]]> + + + 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -648,40 +2193,53 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) == Optimized Physical Plan == -Calc(select=[user_id_0, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, price, user_id_2], where=[>(price, 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == -Calc(select=[user_id_0, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id_0,name,cash,order_id,user_id_1,product,payment_id,price,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2)]) - :- Exchange(distribution=[hash[user_id_0]]) - : +- ChangelogNormalize(key=[user_id_0], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id_0]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id_0, name, cash]) - :- Exchange(distribution=[hash[user_id_1]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id_1, product]) - +- Exchange(distribution=[hash[user_id_2]]) - +- Calc(select=[payment_id, price, user_id_2], where=[(price > 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) ++- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) + :- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[user_id]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + :- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + +- Exchange(distribution=[hash[user_id]]) + +- Calc(select=[payment_id, user_id], where=[(price > 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - + - 10]]> + 10]]> ($7, 10))]) +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) @@ -690,81 +2248,312 @@ LogicalProject(user_id_0=[$0], name=[$1], order_id=[$3], payment_id=[$6]) +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) ]]> - + + (price, 10)]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) +]]> + + + + + + + + + + 10)]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id_2]) +Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[id], orderBy=[$rowtime DESC], select=[id, val, $rowtime]) ++- Exchange(distribution=[hash[id]]) + +- Calc(select=[id, val, $rowtime]) + +- MultiJoin(commonJoinKey=[id], joinTypes=[INNER], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(id, id0)], select=[id,val,$rowtime,id0,price,$rowtime0], rowType=[RecordType(VARCHAR(2147483647) id, INTEGER val, TIMESTAMP(3) $rowtime, VARCHAR(2147483647) id0, DOUBLE price, TIMESTAMP(3) $rowtime0)]) + :- Exchange(distribution=[hash[id]]) + : +- Calc(select=[id, val, CAST($rowtime AS TIMESTAMP(3)) AS $rowtime]) + : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) + +- Exchange(distribution=[hash[id]]) + +- Calc(select=[id, price, CAST($rowtime AS TIMESTAMP(3)) AS $rowtime]) + +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) ]]> - + - + - + + + + + + + + + + + + + + + + + 100]]> + + + (-($1, $2), 100)]) + +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], promo_id=[$7], product_key0=[$8], promo_text=[$9]) + +- LogicalJoin(condition=[=($6, $8)], joinType=[left]) + :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], $f6=[-($1, $2)]) + : +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$4], product_key=[$5], quantity=[$6]) + : +- LogicalJoin(condition=[=($3, $5)], joinType=[left]) + : :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], $f3=[-($1, $2)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, Products]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) + +- LogicalTableScan(table=[[default_catalog, default_database, Promotions]]) +]]> + + + (-(price, discount), 100)]) + : : +- Exchange(distribution=[hash[product_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) + : +- Exchange(distribution=[hash[product_key]]) + : +- Calc(select=[product_key, quantity]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + +- Exchange(distribution=[hash[product_key]]) + +- Calc(select=[product_key, promo_text]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java index 9566dd91186f7..10a2a9f3e0cc9 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/keyselector/AttributeBasedJoinKeyExtractor.java @@ -22,6 +22,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.NoCommonJoinKeyException; import javax.annotation.Nullable; @@ -420,7 +421,7 @@ private void processCommonAttributes(List> commonConceptualAtt if (commonAttrsForThisInput.isEmpty()) { // This indicates that there is no common join key among all inputs. // In this case, we cannot use a multi-join, so throw an exception. - throw new IllegalStateException( + throw new NoCommonJoinKeyException( "All inputs in a multi-way join must share a common join key. Input #" + currentInputId + " does not share a join key with the other inputs. Please ensure all join" From aa6838d297cfb930a4eb5ac6c2955c69fc919687 Mon Sep 17 00:00:00 2001 From: Gustavo de Morais Date: Wed, 13 Aug 2025 08:00:27 +0100 Subject: [PATCH 5/6] [FLINK-38230][table-planner] Add visitMultiJoin to StreamNDUPlanVisitor This closes #26894. --- .../stream/StreamPhysicalMultiJoin.java | 76 +- ...reamNonDeterministicUpdatePlanVisitor.java | 100 + .../NonDeterministicUpdateAnalyzerTest.java | 436 ++-- .../exec/stream/MultiJoinSemanticTests.java | 10 + .../exec/stream/MultiJoinTestPrograms.java | 15 - .../exec/testutils/SemanticTestBase.java | 10 + .../NonDeterministicUpdateAnalyzerTest.xml | 36 + .../planner/plan/stream/sql/MultiJoinTest.xml | 2190 +++-------------- ...ay-complex-updating-join-with-restore.json | 527 ++-- .../savepoint/_metadata | Bin 35375 -> 36360 bytes ...-join-no-common-join-key-with-restore.json | 431 ++-- .../savepoint/_metadata | Bin 38180 -> 34081 bytes .../three-way-inner-join-with-restore.json | 244 +- .../savepoint/_metadata | Bin 21331 -> 20195 bytes ...oin-with-time-attributes-with-restore.json | 89 +- .../savepoint/_metadata | Bin 23917 -> 25432 bytes ...hree-way-left-outer-join-with-restore.json | 231 +- .../savepoint/_metadata | Bin 21588 -> 21005 bytes 18 files changed, 1689 insertions(+), 2706 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java index fb25cab268f3b..a4907e92d0cad 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMultiJoin.java @@ -79,6 +79,10 @@ public class StreamPhysicalMultiJoin extends AbstractRelNode implements StreamPh private final @Nullable RexNode postJoinFilter; private final List hints; + // Cached derived properties to avoid recomputation + private @Nullable RexNode multiJoinCondition; + private @Nullable List> inputUniqueKeys; + public StreamPhysicalMultiJoin( final RelOptCluster cluster, final RelTraitSet traitSet, @@ -101,6 +105,8 @@ public StreamPhysicalMultiJoin( this.postJoinFilter = postJoinFilter; this.hints = hints; this.keyExtractor = keyExtractor; + this.multiJoinCondition = getMultiJoinCondition(); + this.inputUniqueKeys = getUniqueKeysForInputs(); } @Override @@ -119,6 +125,9 @@ public void replaceInput(final int ordinalInParent, final RelNode p) { final List newInputs = new ArrayList<>(inputs); newInputs.set(ordinalInParent, p); this.inputs = List.copyOf(newInputs); + // Invalidate cached derived properties since inputs changed + this.multiJoinCondition = null; + this.inputUniqueKeys = null; recomputeDigest(); } @@ -166,8 +175,8 @@ protected RelDataType deriveRowType() { @Override public ExecNode translateToExecNode() { - final RexNode multiJoinCondition = createMultiJoinCondition(); - final List> inputUniqueKeys = getUniqueKeysForInputs(); + final RexNode multijoinCondition = getMultiJoinCondition(); + final List> localInputUniqueKeys = getUniqueKeysForInputs(); final List execJoinTypes = getExecJoinTypes(); final List inputProperties = createInputProperties(); @@ -175,9 +184,9 @@ public ExecNode translateToExecNode() { unwrapTableConfig(this), execJoinTypes, joinConditions, - multiJoinCondition, + multijoinCondition, joinAttributeMap, - inputUniqueKeys, + localInputUniqueKeys, Collections.emptyMap(), // TODO Enable hint-based state ttl. See ticket // TODO https://issues.apache.org/jira/browse/FLINK-37936 inputProperties, @@ -187,28 +196,43 @@ public ExecNode translateToExecNode() { private RexNode createMultiJoinCondition() { final List conjunctions = new ArrayList<>(); + + for (RexNode joinCondition : joinConditions) { + if (joinCondition != null) { + conjunctions.add(joinCondition); + } + } + conjunctions.add(joinFilter); + if (postJoinFilter != null) { conjunctions.add(postJoinFilter); } + return RexUtil.composeConjunction(getCluster().getRexBuilder(), conjunctions, true); } - private List> getUniqueKeysForInputs() { - return inputs.stream() - .map( - input -> { - final Set uniqueKeys = getUniqueKeys(input); - - if (uniqueKeys == null) { - return Collections.emptyList(); - } - - return uniqueKeys.stream() - .map(ImmutableBitSet::toArray) - .collect(Collectors.toList()); - }) - .collect(Collectors.toList()); + public List> getUniqueKeysForInputs() { + if (inputUniqueKeys == null) { + final List> computed = + inputs.stream() + .map( + input -> { + final Set uniqueKeys = + getUniqueKeys(input); + + if (uniqueKeys == null) { + return Collections.emptyList(); + } + + return uniqueKeys.stream() + .map(ImmutableBitSet::toArray) + .collect(Collectors.toList()); + }) + .collect(Collectors.toList()); + inputUniqueKeys = Collections.unmodifiableList(computed); + } + return inputUniqueKeys; } private @Nullable Set getUniqueKeys(RelNode input) { @@ -217,6 +241,13 @@ private List> getUniqueKeysForInputs() { return fmq.getUniqueKeys(input); } + public RexNode getMultiJoinCondition() { + if (multiJoinCondition == null) { + multiJoinCondition = createMultiJoinCondition(); + } + return multiJoinCondition; + } + private List getExecJoinTypes() { return joinTypes.stream() .map( @@ -256,8 +287,8 @@ public List getJoinTypes() { */ public boolean inputUniqueKeyContainsCommonJoinKey(int inputId) { final RelNode input = getInputs().get(inputId); - final Set inputUniqueKeys = getUniqueKeys(input); - if (inputUniqueKeys == null || inputUniqueKeys.isEmpty()) { + final Set inputUniqueKeysSet = getUniqueKeys(input); + if (inputUniqueKeysSet == null || inputUniqueKeysSet.isEmpty()) { return false; } @@ -267,7 +298,8 @@ public boolean inputUniqueKeyContainsCommonJoinKey(int inputId) { } final ImmutableBitSet commonJoinKeys = ImmutableBitSet.of(commonJoinKeyIndices); - return inputUniqueKeys.stream().anyMatch(uniqueKey -> uniqueKey.contains(commonJoinKeys)); + return inputUniqueKeysSet.stream() + .anyMatch(uniqueKey -> uniqueKey.contains(commonJoinKeys)); } private List createInputProperties() { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java index b1f4d62202731..875a088ec591f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java @@ -45,6 +45,7 @@ import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregateBase; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; @@ -77,6 +78,7 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.JoinRelType; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexProgram; @@ -180,6 +182,8 @@ public StreamPhysicalRel visit( return visitExpand((StreamPhysicalExpand) rel, requireDeterminism); } else if (rel instanceof CommonPhysicalJoin) { return visitJoin((CommonPhysicalJoin) rel, requireDeterminism); + } else if (rel instanceof StreamPhysicalMultiJoin) { + return visitMultiJoin((StreamPhysicalMultiJoin) rel, requireDeterminism); } else if (rel instanceof StreamPhysicalOverAggregateBase) { return visitOverAggregate((StreamPhysicalOverAggregateBase) rel, requireDeterminism); } else if (rel instanceof StreamPhysicalRank) { @@ -621,6 +625,102 @@ private StreamPhysicalRel visitJoin( join.isSemiJoin()); } + /** + * Multi-join determinism handling, mirroring the binary join logic: + * + *

If all inputs are insert-only and every join is INNER, the output is insert-only → no + * determinism required downstream. + * + *

Otherwise the combined join condition must be deterministic, and we propagate per-input + * determinism: + * + *

    + *
  • If an input can produce updates, and we cannot guarantee uniqueness, we must require + * determinism for the entire input row (retract-by-row correctness). + *
  • If uniqueness is guaranteed, we pass through the part of the requirement that belongs + * to that input. + *
+ */ + private StreamPhysicalRel visitMultiJoin( + final StreamPhysicalMultiJoin multiJoin, final ImmutableBitSet requireDeterminism) { + final List inputs = multiJoin.getInputs(); + final boolean allInputsInsertOnly = + inputs.stream().allMatch(in -> inputInsertOnly((StreamPhysicalRel) in)); + final boolean allInner = + multiJoin.getJoinTypes().stream().allMatch(t -> t == JoinRelType.INNER); + + // Fast path: pure insert-only inner join produces insert-only output -> nothing to require. + if (allInputsInsertOnly && allInner) { + return transmitDeterminismRequirement(multiJoin, NO_REQUIRED_DETERMINISM); + } + + // Output may carry updates (some input updates or some non-inner join): condition must be + // deterministic. + final RexNode multiJoinCondition = multiJoin.getMultiJoinCondition(); + if (multiJoinCondition != null) { + final Optional ndCall = + FlinkRexUtil.getNonDeterministicCallName(multiJoinCondition); + ndCall.ifPresent( + s -> throwNonDeterministicConditionError(s, multiJoinCondition, multiJoin)); + } + + // Output may carry updates: we need to propagate determinism requirements to inputs. + final List newInputs = rewriteMultiJoinInputs(multiJoin, requireDeterminism); + + return (StreamPhysicalRel) multiJoin.copy(multiJoin.getTraitSet(), newInputs); + } + + private ImmutableBitSet projectToInput( + final ImmutableBitSet globalRequired, final int inputStart, final int inputFieldCount) { + final List local = + globalRequired.toList().stream() + .filter(idx -> idx >= inputStart && idx < inputStart + inputFieldCount) + .map(idx -> idx - inputStart) + .collect(Collectors.toList()); + return ImmutableBitSet.of(local); + } + + private ImmutableBitSet requiredForUpdatingMultiJoinInput( + final StreamPhysicalMultiJoin multiJoin, + final int inputIndex, + final ImmutableBitSet localRequired, + final int inputFieldCount) { + final List uniqueKeys = multiJoin.getUniqueKeysForInputs().get(inputIndex); + final boolean hasUniqueKey = !uniqueKeys.isEmpty(); + + if (hasUniqueKey) { + return localRequired; + } + // Without uniqueness guarantees we must retract by entire row for correctness. + return ImmutableBitSet.range(inputFieldCount); + } + + private List rewriteMultiJoinInputs( + final StreamPhysicalMultiJoin multiJoin, final ImmutableBitSet requireDeterminism) { + final List inputs = multiJoin.getInputs(); + final List newInputs = new ArrayList<>(inputs.size()); + int fieldStartOffset = 0; + for (int i = 0; i < inputs.size(); i++) { + final StreamPhysicalRel input = (StreamPhysicalRel) inputs.get(i); + final int inputFieldCount = input.getRowType().getFieldCount(); + + final ImmutableBitSet localRequired = + projectToInput(requireDeterminism, fieldStartOffset, inputFieldCount); + + final ImmutableBitSet inputRequired = + inputInsertOnly(input) + ? NO_REQUIRED_DETERMINISM + : requiredForUpdatingMultiJoinInput( + multiJoin, i, localRequired, inputFieldCount); + + final ImmutableBitSet finalRequired = + requireDeterminismExcludeUpsertKey(input, inputRequired); + newInputs.add(visit(input, finalRequired)); + fieldStartOffset += inputFieldCount; + } + return newInputs; + } + private StreamPhysicalRel visitOverAggregate( final StreamPhysicalOverAggregateBase overAgg, final ImmutableBitSet requireDeterminism) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java index 045130d0814eb..99cc20b60da64 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.java @@ -21,6 +21,7 @@ import org.apache.flink.table.api.ExplainDetail; import org.apache.flink.table.api.StatementSet; import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.api.config.OptimizerConfigOptions.NonDeterministicUpdateStrategy; @@ -34,6 +35,7 @@ import scala.Enumeration; +import static org.apache.flink.table.api.config.OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static scala.runtime.BoxedUnit.UNIT; @@ -41,6 +43,7 @@ /** Test for {@link NonDeterministicUpdateAnalyzer}. */ class NonDeterministicUpdateAnalyzerTest extends TableTestBase { + private TableEnvironment tEnv; private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); private static final String expectedOverAggNduErrorMsg = @@ -52,67 +55,63 @@ class NonDeterministicUpdateAnalyzerTest extends TableTestBase { @BeforeEach void before() { - util.getTableEnv() - .executeSql( - "create temporary table cdc (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d bigint,\n" - + " primary key (a) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table cdc_with_meta (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d boolean,\n" - + " metadata_1 int metadata,\n" - + " metadata_2 string metadata,\n" - + " metadata_3 bigint metadata,\n" - + " primary key (a) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D',\n" - + " 'readable-metadata' = 'metadata_1:INT, metadata_2:STRING, metadata_3:BIGINT'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table sink_with_pk (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " primary key (a) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table sink_without_pk (\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table dim_with_pk (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " primary key (a) not enforced\n" - + ") with (\n" - + " 'connector' = 'values'\n" - + ")"); + tEnv = util.getTableEnv(); + tEnv.executeSql( + "create temporary table cdc (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d bigint,\n" + + " primary key (a) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + tEnv.executeSql( + "create temporary table cdc_with_meta (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d boolean,\n" + + " metadata_1 int metadata,\n" + + " metadata_2 string metadata,\n" + + " metadata_3 bigint metadata,\n" + + " primary key (a) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D',\n" + + " 'readable-metadata' = 'metadata_1:INT, metadata_2:STRING, metadata_3:BIGINT'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink_with_pk (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " primary key (a) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink_without_pk (\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary table dim_with_pk (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " primary key (a) not enforced\n" + + ") with (\n" + + " 'connector' = 'values'\n" + + ")"); // custom ND function - util.getTableEnv().createTemporaryFunction("ndFunc", new TestNonDeterministicUdf()); + tEnv.createTemporaryFunction("ndFunc", new TestNonDeterministicUdf()); String sourceTable = "CREATE TABLE source_t(\n" @@ -123,7 +122,7 @@ void before() { + " 'connector' = 'values',\n" + " 'bounded' = 'false',\n" + " 'changelog-mode' = 'I,UB,UA')"; - util.getTableEnv().executeSql(sourceTable); + tEnv.executeSql(sourceTable); String sinkTable = "CREATE TABLE sink_t(\n" @@ -136,39 +135,37 @@ void before() { + " 'bounded' = 'false',\n" + " 'sink-insert-only' = 'false',\n" + " 'changelog-mode' = 'I,UB,UA')"; - util.getTableEnv().executeSql(sinkTable); + tEnv.executeSql(sinkTable); } @Test void testCdcWithMetaRenameSinkWithCompositePk() { // from NonDeterministicDagTest#testCdcWithMetaRenameSinkWithCompositePk - util.getTableEnv() - .executeSql( - "create temporary table cdc_with_meta_rename (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d boolean,\n" - + " metadata_3 bigint metadata,\n" - + " e as metadata_3,\n" - + " primary key (a) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D',\n" - + " 'readable-metadata' = 'metadata_3:BIGINT'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table sink_with_composite_pk (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d bigint,\n" - + " primary key (a,d) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false'\n" - + ")"); + tEnv.executeSql( + "create temporary table cdc_with_meta_rename (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d boolean,\n" + + " metadata_3 bigint metadata,\n" + + " e as metadata_3,\n" + + " primary key (a) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D',\n" + + " 'readable-metadata' = 'metadata_3:BIGINT'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink_with_composite_pk (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d bigint,\n" + + " primary key (a,d) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); util.doVerifyPlanInsert( "insert into sink_with_composite_pk\n" + "select a, b, c, e from cdc_with_meta_rename", @@ -180,20 +177,19 @@ void testCdcWithMetaRenameSinkWithCompositePk() { @Test void testCdcSourceWithoutPkSinkWithoutPk() { // from NonDeterministicDagTest#testCdcSourceWithoutPkSinkWithoutPk - util.getTableEnv() - .executeSql( - "create temporary table cdc_without_pk (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d boolean,\n" - + " metadata_1 int metadata,\n" - + " metadata_2 string metadata\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D',\n" - + " 'readable-metadata' = 'metadata_1:INT, metadata_2:STRING'\n" - + ")"); + tEnv.executeSql( + "create temporary table cdc_without_pk (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d boolean,\n" + + " metadata_1 int metadata,\n" + + " metadata_2 string metadata\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D',\n" + + " 'readable-metadata' = 'metadata_1:INT, metadata_2:STRING'\n" + + ")"); util.doVerifyPlanInsert( "insert into sink_without_pk\n" @@ -207,19 +203,18 @@ void testCdcSourceWithoutPkSinkWithoutPk() { @Test void testSourceWithComputedColumnSinkWithPk() { // from NonDeterministicDagTest#testSourceWithComputedColumnSinkWithPk - util.getTableEnv() - .executeSql( - "create temporary table cdc_with_computed_col (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d int,\n" - + " `day` as DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd'),\n" - + " primary key(a, c) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D'\n" - + ")"); + tEnv.executeSql( + "create temporary table cdc_with_computed_col (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " `day` as DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd'),\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); util.doVerifyPlanInsert( "insert into sink_with_pk\n" + "select a, b, `day`\n" @@ -274,68 +269,63 @@ void testGroupByNonDeterministicFuncWithCdcSource() { @Test void testMultiSinkOnJoinedView() { // from NonDeterministicDagTest#testMultiSinkOnJoinedView - util.getTableEnv() - .executeSql( - "create temporary table src1 (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d int,\n" - + " primary key(a, c) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table src2 (\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " d int,\n" - + " primary key(a, c) not enforced\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'changelog-mode' = 'I,UA,UB,D'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table sink1 (\n" - + " a int,\n" - + " b string,\n" - + " c bigint,\n" - + " d bigint\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary table sink2 (\n" - + " a int,\n" - + " b string,\n" - + " c bigint,\n" - + " d string\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false'\n" - + ")"); - util.getTableEnv() - .executeSql( - "create temporary view v1 as\n" - + "select\n" - + " t1.a as a, t1.`day` as `day`, t2.b as b, t2.c as c\n" - + "from (\n" - + " select a, b, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') as `day`\n" - + " from src1\n" - + " ) t1\n" - + "join (\n" - + " select b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `day`, c, d\n" - + " from src2\n" - + ") t2\n" - + " on t1.a = t2.d"); - - StatementSet stmtSet = util.getTableEnv().createStatementSet(); + tEnv.executeSql( + "create temporary table src1 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + tEnv.executeSql( + "create temporary table src2 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink1 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d bigint\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink2 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d string\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary view v1 as\n" + + "select\n" + + " t1.a as a, t1.`day` as `day`, t2.b as b, t2.c as c\n" + + "from (\n" + + " select a, b, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') as `day`\n" + + " from src1\n" + + " ) t1\n" + + "join (\n" + + " select b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `day`, c, d\n" + + " from src2\n" + + ") t2\n" + + " on t1.a = t2.d"); + + StatementSet stmtSet = tEnv.createStatementSet(); stmtSet.addInsertSql( "insert into sink1\n" + " select a, `day`, sum(b), count(distinct c)\n" @@ -356,6 +346,87 @@ void testMultiSinkOnJoinedView() { false); } + @Test + void testMultiSinkOnMultiJoinedView() { + tEnv.getConfig().set(TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true); + tEnv.executeSql( + "create temporary table src1 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + tEnv.executeSql( + "create temporary table src2 (\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " d int,\n" + + " primary key(a, c) not enforced\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink1 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d bigint\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary table sink2 (\n" + + " a int,\n" + + " b string,\n" + + " c bigint,\n" + + " d string\n" + + ") with (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'false'\n" + + ")"); + tEnv.executeSql( + "create temporary view v1 as\n" + + "select\n" + + " t1.a as a, t1.`day` as `day`, t2.b as b, t2.c as c\n" + + "from (\n" + + " select a, b, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd') as `day`\n" + + " from src1\n" + + " ) t1\n" + + "join (\n" + + " select b, CONCAT(c, DATE_FORMAT(CURRENT_TIMESTAMP, 'yyMMdd')) as `day`, c, d\n" + + " from src2\n" + + ") t2\n" + + " on t1.a = t2.d"); + + StatementSet stmtSet = tEnv.createStatementSet(); + stmtSet.addInsertSql( + "insert into sink1\n" + + " select a, `day`, sum(b), count(distinct c)\n" + + " from v1\n" + + " group by a, `day`"); + stmtSet.addInsertSql( + "insert into sink2\n" + + " select a, `day`, b, c\n" + + " from v1\n" + + " where b > 100"); + + util.doVerifyPlan( + stmtSet, + new ExplainDetail[] {ExplainDetail.PLAN_ADVICE}, + false, + new Enumeration.Value[] {PlanKind.OPT_REL_WITH_ADVICE()}, + () -> UNIT, + false, + false); + } + @Test void testCdcJoinDimWithPkOutputNoPkSinkWithoutPk() { // from NonDeterministicDagTest#testCdcJoinDimWithPkOutputNoPkSinkWithoutPk @@ -373,8 +444,7 @@ void testCdcJoinDimWithPkOutputNoPkSinkWithoutPk() { @Test void testOverAggregateWithNonDeterminismInPartitionBy() { - util.getTableEnv() - .getConfig() + tEnv.getConfig() .set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, NonDeterministicUpdateStrategy.TRY_RESOLVE); @@ -395,8 +465,7 @@ void testOverAggregateWithNonDeterminismInPartitionBy() { @Test void testOverAggregateWithNonDeterminismInOrderBy() { - util.getTableEnv() - .getConfig() + tEnv.getConfig() .set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, NonDeterministicUpdateStrategy.TRY_RESOLVE); @@ -417,8 +486,7 @@ void testOverAggregateWithNonDeterminismInOrderBy() { @Test void testOverAggregateWithNonDeterminismInProjection() { - util.getTableEnv() - .getConfig() + tEnv.getConfig() .set( OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, NonDeterministicUpdateStrategy.TRY_RESOLVE); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java index fb24e92e1d736..d9f371bcd47e7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinSemanticTests.java @@ -18,6 +18,8 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.planner.plan.nodes.exec.testutils.SemanticTestBase; import org.apache.flink.table.test.program.TableTestProgram; @@ -26,6 +28,14 @@ /** Semantic tests for {@link StreamExecMultiJoin}. */ public class MultiJoinSemanticTests extends SemanticTestBase { + @Override + protected void applyDefaultEnvironmentOptions(TableConfig config) { + config.set( + OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, + OptimizerConfigOptions.NonDeterministicUpdateStrategy.TRY_RESOLVE) + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true); + } + @Override public List programs() { return List.of( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java index 3e8e27e9c305f..2137a1e5642d3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MultiJoinTestPrograms.java @@ -36,7 +36,6 @@ public class MultiJoinTestPrograms { public static final TableTestProgram MULTI_JOIN_THREE_WAY_LEFT_OUTER_JOIN = TableTestProgram.of("three-way-left-outer-join", "three way left outer join") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource(USERS_SOURCE) .setupTableSource(ORDERS_SOURCE) .setupTableSource(PAYMENTS_SOURCE) @@ -67,7 +66,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-left-outer-join-updating", "three way left outer join updating") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema( @@ -110,7 +108,6 @@ public class MultiJoinTestPrograms { public static final TableTestProgram MULTI_JOIN_THREE_WAY_INNER_JOIN = TableTestProgram.of("three-way-inner-join", "three way inner join") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource(USERS_SOURCE) .setupTableSource(ORDERS_SOURCE) .setupTableSource(PAYMENTS_SOURCE) @@ -140,7 +137,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-inner-join-with-where", "three way inner join with where clause") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource(USERS_SOURCE) .setupTableSource(ORDERS_SOURCE) .setupTableSource(PAYMENTS_SOURCE) @@ -169,7 +165,6 @@ public class MultiJoinTestPrograms { public static final TableTestProgram MULTI_JOIN_FOUR_WAY_COMPLEX = TableTestProgram.of("four-way-complex-updating-join", "four way complex updating join") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema( @@ -281,7 +276,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-left-outer-join-with-restore", "three way left outer join with restore") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema("user_id STRING", "name STRING") @@ -335,7 +329,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-inner-join-with-restore", "three way inner join with restore") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema("user_id STRING", "name STRING") @@ -388,7 +381,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-inner-join-no-join-key", "three way inner join with no join key") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource(USERS_SOURCE) .setupTableSource(ORDERS_SOURCE) .setupTableSource(PAYMENTS_SOURCE) @@ -462,7 +454,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "four-way-join-no-common-join-key-with-restore", "four way join no common join key with restore") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupConfig(TableConfigOptions.PLAN_FORCE_RECOMPILE, true) .setupTableSource( SourceTestStep.newBuilder("Users") @@ -572,7 +563,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "four-way-no-common-join-key-updating-join", "four way no common join key updating join") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("Users") .addSchema( @@ -675,7 +665,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "four-way-complex-updating-join-with-restore", "four way complex updating join with restore") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupConfig(TableConfigOptions.PLAN_FORCE_RECOMPILE, true) .setupTableSource( SourceTestStep.newBuilder("Users") @@ -784,7 +773,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-join-with-time-attributes", "three way join with time attributes materialization") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource( SourceTestStep.newBuilder("UsersWithProctime") .addSchema( @@ -912,7 +900,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "left-outer-join-with-cte", "CTE with three-way left outer join and aggregation") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupTableSource(USERS_SOURCE) .setupTableSource( SourceTestStep.newBuilder("Orders") @@ -976,7 +963,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "three-way-left-outer-with-null-keys", "left outer join with NULL keys on multiple inputs") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupSql( "CREATE VIEW UsersNulls AS SELECT * FROM (VALUES " + "('1','Gus')," @@ -1018,7 +1004,6 @@ public class MultiJoinTestPrograms { TableTestProgram.of( "null-safe-join-with-null-keys", "join with IS NOT DISTINCT FROM to match NULL keys") - .setupConfig(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTI_JOIN_ENABLED, true) .setupSql( "CREATE VIEW UsersNullSafe AS SELECT * FROM (VALUES " + "('1','Gus')," diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/SemanticTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/SemanticTestBase.java index ac3b0a139392e..5e72649ac7e9c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/SemanticTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/SemanticTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.testutils; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.table.test.program.ConfigOptionTestStep; @@ -81,6 +82,8 @@ public void clearData() { void runSteps(TableTestProgram program) throws Exception { final TableEnvironment env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + applyDefaultEnvironmentOptions(env.getConfig()); + for (TestStep testStep : program.setupSteps) { runStep(testStep, env); } @@ -149,6 +152,13 @@ protected void runStep(TestStep testStep, TableEnvironment env) throws Exception } } + /** + * Hook for subclasses to apply suite-wide default table configuration options. + * + *

Default implementation is a no-op. Subclasses can override to set specific options. + */ + protected void applyDefaultEnvironmentOptions(TableConfig config) {} + private static Map createSourceOptions( SourceTestStep sourceTestStep, String id) { final Map options = new HashMap<>(); diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.xml index dc45a0b5c9dc9..7ee2cc9a66eca 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/analyze/NonDeterministicUpdateAnalyzerTest.xml @@ -203,6 +203,42 @@ Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], cha +- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA,D]) +]]> + + + + + (b, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). +advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. + +related rel plan: +Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) ++- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b], changelogMode=[I,UB,UA,D]) + + ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 6c3a455d789a5..56bf1998204f0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -17,201 +17,78 @@ limitations under the License. --> - - 5 THEN 1 END) as bulk_orders -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN OrderItems oi - ON o.order_id = oi.order_id -LEFT JOIN ProductCategories pc - ON oi.product_name = pc.category_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -GROUP BY u.user_id, u.name, pc.category_name -HAVING COUNT(DISTINCT o.order_id) > 0]]> - - - ($3, 0)]) -+- LogicalAggregate(group=[{0, 1, 2}], order_count=[COUNT(DISTINCT $3)], total_items=[SUM($4)], total_value=[SUM($5)], avg_item_price=[AVG($6)], max_payment=[MAX($7)], min_payment=[MIN($7)], bulk_orders=[COUNT($8)]) - +- LogicalProject(user_id=[$0], name=[$1], category_name=[$12], order_id=[$3], quantity=[$9], $f5=[*($9, $10)], unit_price=[$10], price=[$15], $f8=[CASE(>($9, 5), 1, null:INTEGER)]) - +- LogicalJoin(condition=[=($0, $16)], joinType=[left]) - :- LogicalJoin(condition=[=($8, $11)], joinType=[left]) - : :- LogicalJoin(condition=[=($3, $7)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, OrderItems]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (order_count, 0)]) +- GroupAggregate(groupBy=[user_id, name, category_name], select=[user_id, name, category_name, COUNT_RETRACT(DISTINCT order_id) AS order_count, SUM_RETRACT(quantity) AS total_items, SUM_RETRACT($f5) AS total_value, AVG_RETRACT(unit_price) AS avg_item_price, MAX_RETRACT(price) AS max_payment, MIN_RETRACT(price) AS min_payment, COUNT_RETRACT($f8) AS bulk_orders]) +- Exchange(distribution=[hash[user_id, name, category_name]]) +- Calc(select=[user_id, name, category_name, order_id, quantity, *(quantity, unit_price) AS $f5, unit_price, price, CASE(>(quantity, 5), 1, null:INTEGER) AS $f8]) - +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id1)], select=[user_id,name,order_id,quantity,unit_price,category_name,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_name, INTEGER price, VARCHAR(2147483647) user_id1)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($8, $11), =($0, $16)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:8;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:8;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price,category_id,category_name,parent_category,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, name, order_id, quantity, unit_price, category_name]) - : +- MultiJoin(commonJoinKey=[product_name], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (category_id)], joinConditions=[=(product_name, category_id)], select=[user_id,name,order_id,product_name,quantity,unit_price,category_id,category_name], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name)]) - : :- Exchange(distribution=[hash[product_name]]) - : : +- Calc(select=[user_id, name, order_id, product_name, quantity, unit_price]) - : : +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) - : : :- Exchange(distribution=[hash[order_id]]) - : : : +- Calc(select=[user_id, name, order_id]) - : : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) - : : : :- Exchange(distribution=[hash[user_id]]) - : : : : +- ChangelogNormalize(key=[user_id]) - : : : : +- Exchange(distribution=[hash[user_id]]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) - : : +- Exchange(distribution=[hash[order_id]]) - : : +- Calc(select=[order_id, product_name, quantity, unit_price]) - : : +- ChangelogNormalize(key=[item_id]) - : : +- Exchange(distribution=[hash[item_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) - : +- Exchange(distribution=[hash[category_id]]) - : +- ChangelogNormalize(key=[category_id]) - : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories, project=[category_id, category_name], metadata=[]]], fields=[category_id, category_name]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($3, $7)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:3;], 1=[LeftInputId:0;LeftFieldIndex:3;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,item_id,order_id0,product_name,quantity,unit_price], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) item_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) product_name, INTEGER quantity, DOUBLE unit_price)]) + : :- Exchange(distribution=[hash[order_id]]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[order_id]]) + : +- ChangelogNormalize(key=[item_id]) + : +- Exchange(distribution=[hash[item_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, OrderItems]], fields=[item_id, order_id, product_name, quantity, unit_price]) + :- Exchange(distribution=[hash[category_id]]) + : +- ChangelogNormalize(key=[category_id]) + : +- Exchange(distribution=[hash[category_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, parent_category]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - 0]]> - - - ($2, 0)]) -+- LogicalAggregate(group=[{0}], unique_users=[COUNT(DISTINCT $1)], total_sales=[COUNT($2)], total_revenue=[SUM($3)], avg_sale_amount=[AVG($3)], max_sale_amount=[MAX($3)]) - +- LogicalProject(category_name=[$7], user_id=[$0], sale_id=[$10], amount=[$13]) - +- LogicalJoin(condition=[=($0, $11)], joinType=[left]) - :- LogicalJoin(condition=[AND(=($0, $9), =($5, $6))], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Categories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) -]]> - (total_sales, 0)]) +- GroupAggregate(groupBy=[category_name], select=[category_name, COUNT_RETRACT(DISTINCT user_id) AS unique_users, COUNT_RETRACT(sale_id) AS total_sales, SUM_RETRACT(amount) AS total_revenue, AVG_RETRACT(amount) AS avg_sale_amount, MAX_RETRACT(amount) AS max_sale_amount]) +- Exchange(distribution=[hash[category_name]]) +- Calc(select=[category_name, user_id, sale_id, amount]) - +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, (category_id), (sale_id)], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), =(product, category_id)), =(user_id, user_id2)], select=[user_id,user_id0,product,category_id,category_name,user_id1,sale_id,user_id2,amount], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, DOUBLE amount)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $9), =($5, $6)), =($0, $11)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:3;, LeftInputId:1;LeftFieldIndex:2;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:0;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,category_id,category_name,parent_category,user_id1,sale_id,user_id2,product_id,amount,sale_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, VARCHAR(2147483647) parent_category, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) sale_id, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) product_id, DOUBLE amount, DATE sale_date)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Categories, project=[category_id, category_name, user_id], metadata=[]]], fields=[category_id, category_name, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Categories]], fields=[category_id, category_name, parent_category, user_id]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[sale_id]) +- Exchange(distribution=[hash[sale_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Sales, project=[sale_id, user_id, amount], metadata=[]]], fields=[sale_id, user_id, amount]) + +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, user_id, product_id, amount, sale_date]) ]]> - - 600000 -), -active_projects AS ( - SELECT project_id, project_name, dept_id - FROM Projects - WHERE status = 'ACTIVE' -) -SELECT - u.user_id, - u.name, - o.order_id, - hbd.dept_name, - ap.project_name, - hbd.budget -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN high_budget_depts hbd - ON o.user_id = hbd.dept_id -LEFT JOIN active_projects ap - ON hbd.dept_id = ap.dept_id]]> - - - ($2, 600000)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Departments]]) - +- LogicalProject(project_id=[$0], project_name=[$1], dept_id=[$2]) - +- LogicalFilter(condition=[=($3, _UTF-16LE'ACTIVE')]) - +- LogicalTableScan(table=[[default_catalog, default_database, Projects]]) -]]> - (budget, 600000)]) : +- Exchange(distribution=[hash[dept_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Departments, filter=[]]], fields=[dept_id, dept_name, budget]) +- Exchange(distribution=[hash[dept_id]]) - +- Calc(select=[project_name, dept_id]) + +- Calc(select=[project_id, project_name, dept_id]) +- ChangelogNormalize(key=[project_id], condition=[=(status, 'ACTIVE')]) +- Exchange(distribution=[hash[project_id]]) +- TableSourceScan(table=[[default_catalog, default_database, Projects, filter=[]]], fields=[project_id, project_name, dept_id, status]) @@ -219,131 +96,44 @@ Calc(select=[user_id, name, order_id, dept_name, project_name, budget]) - - - - - - - - - - - - - - - - - - - - 1000 THEN 'High-Value Premium' - WHEN pc.is_premium = true THEN 'Premium' - WHEN p.price > 500 THEN 'Standard High-Value' - ELSE 'Standard' - END AS product_tier, - CASE - WHEN pr.rating >= 4 AND pr.is_verified = true THEN 'Highly Recommended' - WHEN pr.rating >= 3 THEN 'Recommended' - WHEN pr.rating >= 2 THEN 'Average' - ELSE 'Not Recommended' - END AS recommendation_status, - CASE - WHEN pc.discount_rate > 0.2 THEN p.price * (1 - pc.discount_rate) - ELSE p.price - END AS final_price -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -LEFT JOIN ProductCategories pc - ON o.product = pc.category_id -LEFT JOIN ProductReviews pr - ON o.product = pr.product_id]]> - - - ($7, 1000)), _UTF-16LE'High-Value Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", $11, _UTF-16LE'Premium':VARCHAR(19) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Standard High-Value':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Standard':VARCHAR(19) CHARACTER SET "UTF-16LE")], recommendation_status=[CASE(AND(>=($15, 4), $16), _UTF-16LE'Highly Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 3), _UTF-16LE'Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE", >=($15, 2), _UTF-16LE'Average':VARCHAR(18) CHARACTER SET "UTF-16LE", _UTF-16LE'Not Recommended':VARCHAR(18) CHARACTER SET "UTF-16LE")], final_price=[CASE(>($12, 0.2:DECIMAL(2, 1)), *($7, -(1, $12)), CAST($7):DOUBLE)]) -+- LogicalJoin(condition=[=($5, $14)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductCategories]]) - +- LogicalTableScan(table=[[default_catalog, default_database, ProductReviews]]) -]]> - (price, 1000)), 'High-Value Premium', is_premium, 'Premium', >(price, 500), 'Standard High-Value', 'Standard') AS product_tier, CASE(AND(>=(rating, 4), is_verified), 'Highly Recommended', >=(rating, 3), 'Recommended', >=(rating, 2), 'Average', 'Not Recommended') AS recommendation_status, CASE(>(discount_rate, 0.2), *(price, -(1, discount_rate)), CAST(price AS DOUBLE)) AS final_price]) -+- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (category_id), noUniqueKey], joinConditions=[=(product, category_id), =(product, product_id)], select=[user_id,order_id,product,payment_id,price,category_id,category_name,is_premium,discount_rate,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($5, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:5;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,category_id,category_name,is_premium,discount_rate,review_id,product_id,rating,is_verified], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) category_id, VARCHAR(2147483647) category_name, BOOLEAN is_premium, DOUBLE discount_rate, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id, INTEGER rating, BOOLEAN is_verified)]) :- Exchange(distribution=[hash[product]]) - : +- Calc(select=[user_id, order_id, product, payment_id, price]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id], metadata=[]]], fields=[user_id]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) :- Exchange(distribution=[hash[category_id]]) : +- ChangelogNormalize(key=[category_id]) : +- Exchange(distribution=[hash[category_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, ProductCategories]], fields=[category_id, category_name, is_premium, discount_rate]) +- Exchange(distribution=[hash[product_id]]) - +- Calc(select=[product_id, rating, is_verified]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, ProductReviews]], fields=[review_id, product_id, rating, is_verified]) ]]> - - 1000 THEN 'Premium' - WHEN bo.price > 500 THEN 'Standard' - ELSE 'Basic' - END as order_tier - FROM base_orders bo - LEFT JOIN OrderMetrics om - ON bo.order_id = om.order_id -), -aggregated_metrics AS ( - SELECT - user_id, - name, - COUNT(DISTINCT order_id) as total_orders, - SUM(price) as total_spent, - AVG(price) as avg_order_value, - MAX(metric_value) as max_metric, - MIN(metric_value) as min_metric, - COUNT(CASE WHEN order_tier = 'Premium' THEN 1 END) as premium_orders - FROM enriched_orders - GROUP BY user_id, name -) -SELECT - user_id, - UPPER(name) as user_name, - total_orders, - ROUND(total_spent, 2) as total_spent_rounded, - ROUND(avg_order_value, 2) as avg_order_value_rounded, - CONCAT('User: ', name, ' has ', CAST(total_orders AS STRING), ' orders') as summary, - CASE - WHEN total_orders > 10 THEN 'Frequent Customer' - WHEN total_orders > 5 THEN 'Regular Customer' - ELSE 'Occasional Customer' - END as customer_type -FROM aggregated_metrics -WHERE total_spent > 0]]> - - - ($2, 10), _UTF-16LE'Frequent Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", >($2, 5), _UTF-16LE'Regular Customer':VARCHAR(19) CHARACTER SET "UTF-16LE", _UTF-16LE'Occasional Customer':VARCHAR(19) CHARACTER SET "UTF-16LE")]) -+- LogicalFilter(condition=[>($3, 0)]) - +- LogicalAggregate(group=[{0, 1}], total_orders=[COUNT(DISTINCT $2)], total_spent=[SUM($3)], avg_order_value=[AVG($3)], max_metric=[MAX($4)], min_metric=[MIN($4)], premium_orders=[COUNT($5)]) - +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], price=[$4], metric_value=[$6], $f5=[CASE(=($7, _UTF-16LE'Premium'), 1, null:INTEGER)]) - +- LogicalProject(user_id=[$0], name=[$1], order_id=[$2], payment_id=[$3], price=[$4], metric_type=[$7], metric_value=[$8], order_tier=[CASE(>($4, 1000), _UTF-16LE'Premium':VARCHAR(8) CHARACTER SET "UTF-16LE", >($4, 500), _UTF-16LE'Standard':VARCHAR(8) CHARACTER SET "UTF-16LE", _UTF-16LE'Basic':VARCHAR(8) CHARACTER SET "UTF-16LE")]) - +- LogicalJoin(condition=[=($2, $6)], joinType=[left]) - :- LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], price=[$7]) - : +- LogicalJoin(condition=[=($0, $8)], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, OrderMetrics]]) -]]> - (total_orders, 10), 'Frequent Customer', >(total_orders, 5), 'Regular Customer', 'Occasional Customer') AS customer_type], where=[>(total_spent, 0)]) +- GroupAggregate(groupBy=[user_id, name], select=[user_id, name, COUNT_RETRACT(DISTINCT order_id) AS total_orders, SUM_RETRACT(price) AS total_spent, AVG_RETRACT(price) AS avg_order_value]) +- Exchange(distribution=[hash[user_id, name]]) - +- MultiJoin(commonJoinKey=[order_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(order_id, order_id0)], select=[user_id,name,order_id,price,order_id0,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, INTEGER price, VARCHAR(2147483647) order_id0, DOUBLE metric_value)]) + +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($2, $6)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:2;], 1=[LeftInputId:0;LeftFieldIndex:2;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,order_id,payment_id,price,metric_id,order_id0,metric_type,metric_value], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) metric_id, VARCHAR(2147483647) order_id0, VARCHAR(2147483647) metric_type, DOUBLE metric_value)]) :- Exchange(distribution=[hash[order_id]]) - : +- Calc(select=[user_id, name, order_id, price]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, INTEGER price, VARCHAR(2147483647) user_id1)]) + : +- Calc(select=[user_id, name, order_id, payment_id, price]) + : +- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) : :- Exchange(distribution=[hash[user_id]]) : : +- ChangelogNormalize(key=[user_id]) : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) : :- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[order_id]]) - +- Calc(select=[order_id, metric_value]) - +- ChangelogNormalize(key=[metric_id]) - +- Exchange(distribution=[hash[metric_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics, project=[order_id, metric_value, metric_id], metadata=[]]], fields=[order_id, metric_value, metric_id]) + +- ChangelogNormalize(key=[metric_id]) + +- Exchange(distribution=[hash[metric_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, OrderMetrics]], fields=[metric_id, order_id, metric_type, metric_value]) ]]> - - - - - - - - = p.price OR p.price < 0) -LEFT JOIN Shipments s - ON p.user_id = s.user_id]]> - - - =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - = price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -698,13 +273,13 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6], location == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[=(user_id, user_id0), AND(=(user_id, user_id1), OR(>=(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -712,13 +287,13 @@ Calc(select=[user_id, name, order_id, payment_id, location]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id, location]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id), noUniqueKey], joinConditions=[(user_id = user_id0), ((user_id = user_id1) AND ((cash >= price) OR (price < 0))), (user_id1 = user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -727,45 +302,16 @@ Calc(select=[user_id, name, order_id, payment_id, location]) - - = p.price OR p.price < 0) -LEFT JOIN Shipments s - ON p.user_id = s.user_id]]> - - - =($2, $7), <($7, 0)))], joinType=[inner]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - =(cash, price), <(price, 0))), =(user_id1, user_id2)], select=[user_id,name,cash,order_id,user_id0,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), OR(>=($2, $7), <($7, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($0, $4), AND(=($0, $8), OR(>=($2, $7), <($7, 0))), =($8, $10)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;], 3=[LeftInputId:2;LeftFieldIndex:2;RightInputId:3;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) :- Exchange(distribution=[hash[user_id]]) : +- ChangelogNormalize(key=[user_id]) : +- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[user_id]]) : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -774,192 +320,69 @@ Calc(select=[user_id, name, order_id, payment_id, location]) - - - - - - - - - - - - - - = FLOOR(p.price) OR p.price < 0) -LEFT JOIN Shipments s - ON p.payment_id = s.location]]> - - - =(FLOOR($2), FLOOR($8)), <($8, 0)))], joinType=[left]) - : :- LogicalProject(user_id=[$0], name=[$1], cash=[$2], order_id=[$3], user_id0=[$4], product=[$5], $f6=[UPPER($1)]) - : : +- LogicalJoin(condition=[=($4, $0)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalProject(payment_id=[$0], price=[$1], user_id=[$2], $f3=[UPPER($0)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - =(FLOOR(cash), FLOOR(price)), <(price, 0)))], select=[user_id,name,cash,order_id,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) -: :- Exchange(distribution=[hash[user_id, $f6]]) -: : +- Calc(select=[user_id, name, cash, order_id, UPPER(name) AS $f6]) -: : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id0, user_id)], select=[user_id,name,cash,order_id,user_id0], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0)]) -: : :- Exchange(distribution=[hash[user_id]]) -: : : +- ChangelogNormalize(key=[user_id]) -: : : +- Exchange(distribution=[hash[user_id]]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) -: : +- Exchange(distribution=[hash[user_id]]) -: : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) -: +- Exchange(distribution=[hash[user_id, $f3]]) -: +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) -: +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) -+- Exchange(distribution=[hash[location]]) - +- TableSourceScan(table=[[default_catalog, default_database, Shipments, project=[location], metadata=[]]], fields=[location]) +Calc(select=[user_id, name, order_id, payment_id, location]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($6, $9)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,location,user_id2], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id2)]) + :- Exchange(distribution=[hash[payment_id]]) + : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, payment_id, price, user_id1]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, AND(=($0, $9), =($6, $10), OR(>=(FLOOR($2), FLOOR($8)), <($8, 0)))]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:6;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:2;, LeftInputId:0;LeftFieldIndex:6;RightInputId:1;RightFieldIndex:3;]}], select=[user_id,name,cash,order_id,user_id0,product,$f6,payment_id,price,user_id1,$f3], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) $f6, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) $f3)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- Calc(select=[user_id, name, cash, order_id, user_id0, product, UPPER(name) AS $f6]) + : : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($4, $0)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : : :- Exchange(distribution=[hash[user_id]]) + : : : +- ChangelogNormalize(key=[user_id]) + : : : +- Exchange(distribution=[hash[user_id]]) + : : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- Calc(select=[payment_id, price, user_id, UPPER(payment_id) AS $f3]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + +- Exchange(distribution=[hash[location]]) + +- TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id]) ]]> - - - - - - - - 1000 THEN 'High' - WHEN p.price > 500 THEN 'Medium' - ELSE 'Low' - END as price_tier, - REGEXP_REPLACE(pd.tags, ',', ' | ') as formatted_tags, - TO_TIMESTAMP_LTZ(pd.created_date, 3) as product_created, - COALESCE(up.preferred_category, 'None') as user_preference, - CASE - WHEN up.notification_level = 'HIGH' THEN 'Frequent Updates' - WHEN up.notification_level = 'MEDIUM' THEN 'Daily Updates' - ELSE 'Weekly Updates' - END as notification_frequency -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN Payments p - ON u.user_id = p.user_id -LEFT JOIN ProductDetails pd - ON o.product = pd.product_id -LEFT JOIN UserPreferences up - ON u.user_id = up.user_id]]> - - - ($7, 1000), _UTF-16LE'High':VARCHAR(6) CHARACTER SET "UTF-16LE", >($7, 500), _UTF-16LE'Medium':VARCHAR(6) CHARACTER SET "UTF-16LE", _UTF-16LE'Low':VARCHAR(6) CHARACTER SET "UTF-16LE")], formatted_tags=[REGEXP_REPLACE($13, _UTF-16LE',', _UTF-16LE' | ')], product_created=[TO_TIMESTAMP_LTZ($12, 3)], user_preference=[COALESCE($15, _UTF-16LE'None')], notification_frequency=[CASE(=($16, _UTF-16LE'HIGH'), _UTF-16LE'Frequent Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", =($16, _UTF-16LE'MEDIUM'), _UTF-16LE'Daily Updates':VARCHAR(16) CHARACTER SET "UTF-16LE", _UTF-16LE'Weekly Updates':VARCHAR(16) CHARACTER SET "UTF-16LE")]) -+- LogicalJoin(condition=[=($0, $14)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $9)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - : : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) - +- LogicalTableScan(table=[[default_catalog, default_database, UserPreferences]]) -]]> - (price, 1000), 'High', >(price, 500), 'Medium', 'Low') AS price_tier, REGEXP_REPLACE(tags, ',', ' | ') AS formatted_tags, TO_TIMESTAMP_LTZ(created_date, 3) AS product_created, COALESCE(preferred_category, 'None') AS user_preference, CASE(=(notification_level, 'HIGH'), 'Frequent Updates', =(notification_level, 'MEDIUM'), 'Daily Updates', 'Weekly Updates') AS notification_frequency]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (user_id)], joinConditions=[=(user_id, user_id2)], select=[user_id,name,product,price,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $9), =($0, $14)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1,product_id,product_name,description,created_date,tags,user_id2,preferred_category,notification_level], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags, VARCHAR(2147483647) user_id2, VARCHAR(2147483647) preferred_category, VARCHAR(2147483647) notification_level)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, name, product, price, description, created_date, tags]) - : +- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT], inputUniqueKeys=[noUniqueKey, (product_id)], joinConditions=[=(product, product_id)], select=[user_id,name,product,price,product_id,description,created_date,tags], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) product_id, VARCHAR(2147483647) description, BIGINT created_date, VARCHAR(2147483647) tags)]) - : :- Exchange(distribution=[hash[product]]) - : : +- Calc(select=[user_id, name, product, price]) - : : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), noUniqueKey, noUniqueKey], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,user_id0,product,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, INTEGER price, VARCHAR(2147483647) user_id1)]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- ChangelogNormalize(key=[user_id]) - : : : +- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : : :- Exchange(distribution=[hash[user_id]]) - : : : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[user_id, product], metadata=[]]], fields=[user_id, product]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- ChangelogNormalize(key=[product_id]) - : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, description, created_date, tags], metadata=[]]], fields=[product_id, description, created_date, tags]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) + :- Exchange(distribution=[hash[product_id]]) + : +- ChangelogNormalize(key=[product_id]) + : +- Exchange(distribution=[hash[product_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, description, created_date, tags]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id]) +- Exchange(distribution=[hash[user_id]]) @@ -1055,92 +427,30 @@ Calc(select=[user_id, UPPER(name) AS user_name_upper, LOWER(product) AS product_ - - = 4 THEN 'High Rating' - WHEN r.rating >= 3 THEN 'Medium Rating' - ELSE 'Low Rating' - END AS rating_category, - TIMESTAMPDIFF(DAY, pd.created_date, CURRENT_DATE) AS days_since_created -FROM Users u -LEFT JOIN Orders o - ON u.user_id = o.user_id -LEFT JOIN ProductDetails pd - ON o.product = pd.product_id -LEFT JOIN Reviews r - ON pd.product_id = r.product_id]]> - - - =($13, 4), _UTF-16LE'High Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", >=($13, 3), _UTF-16LE'Medium Rating':VARCHAR(13) CHARACTER SET "UTF-16LE", _UTF-16LE'Low Rating':VARCHAR(13) CHARACTER SET "UTF-16LE")], days_since_created=[CAST(/INT(Reinterpret(-(CURRENT_DATE, $10)), 86400000)):INTEGER]) -+- LogicalJoin(condition=[=($6, $12)], joinType=[left]) - :- LogicalJoin(condition=[=($5, $6)], joinType=[left]) - : :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, ProductDetails]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Reviews]]) -]]> - =(rating, 4), 'High Rating', >=(rating, 3), 'Medium Rating', 'Low Rating') AS rating_category, CAST(/INT(Reinterpret(-(CURRENT_DATE(), created_date)), 86400000) AS INTEGER) AS days_since_created]) -+- MultiJoin(commonJoinKey=[product], joinTypes=[LEFT, LEFT], inputUniqueKeys=[noUniqueKey, (product_id), noUniqueKey], joinConditions=[=(product, product_id), =(product_id, product_id0)], select=[user_id,name,order_id,product,product_id,product_name,price,created_date,product_id0,rating,review_text], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DATE created_date, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($5, $6), =($6, $12)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:5;], 1=[LeftInputId:0;LeftFieldIndex:5;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:1;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product,product_id,product_name,price,weight,created_date,review_id,product_id0,rating,review_text,review_date], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) product_id, VARCHAR(2147483647) product_name, DOUBLE price, DOUBLE weight, DATE created_date, VARCHAR(2147483647) review_id, VARCHAR(2147483647) product_id0, INTEGER rating, VARCHAR(2147483647) review_text, DATE review_date)]) :- Exchange(distribution=[hash[product]]) - : +- Calc(select=[user_id, name, order_id, product]) - : +- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT], inputUniqueKeys=[(user_id), (order_id)], joinConditions=[=(user_id, user_id0)], select=[user_id,name,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) - : :- Exchange(distribution=[hash[user_id]]) - : : +- ChangelogNormalize(key=[user_id]) - : : +- Exchange(distribution=[hash[user_id]]) - : : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) + : +- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($0, $4)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;]}], select=[user_id,name,cash,order_id,user_id0,product], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product)]) + : :- Exchange(distribution=[hash[user_id]]) + : : +- ChangelogNormalize(key=[user_id]) + : : +- Exchange(distribution=[hash[user_id]]) + : : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) :- Exchange(distribution=[hash[product_id]]) : +- ChangelogNormalize(key=[product_id]) : +- Exchange(distribution=[hash[product_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails, project=[product_id, product_name, price, created_date], metadata=[]]], fields=[product_id, product_name, price, created_date]) + : +- TableSourceScan(table=[[default_catalog, default_database, ProductDetails]], fields=[product_id, product_name, price, weight, created_date]) +- Exchange(distribution=[hash[product_id]]) - +- Calc(select=[product_id, rating, review_text]) - +- ChangelogNormalize(key=[review_id]) - +- Exchange(distribution=[hash[review_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Reviews, project=[product_id, rating, review_text, review_id], metadata=[]]], fields=[product_id, rating, review_text, review_id]) + +- ChangelogNormalize(key=[review_id]) + +- Exchange(distribution=[hash[review_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Reviews]], fields=[review_id, product_id, rating, review_text, review_date]) ]]> - - - - - =($2, -($5, 60000:INTERVAL MINUTE)), <=($2, +($5, 60000:INTERVAL MINUTE)))], joinType=[inner]) - :- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, EventTable1]]) - +- LogicalWatermarkAssigner(rowtime=[$rowtime], watermark=[-($2, 5000:INTERVAL SECOND)]) - +- LogicalTableScan(table=[[default_catalog, default_database, EventTable2]]) -]]> - - - - - - - - - 100) AS u -JOIN (SELECT user_id, order_id, product FROM Orders WHERE product IS NOT NULL) AS o - ON u.user_id = o.user_id -LEFT JOIN (SELECT user_id, price FROM Payments WHERE price > 50) AS p - ON u.user_id = p.user_id -LEFT JOIN (SELECT user_id, location FROM Shipments WHERE location IS NOT NULL) AS s - ON u.user_id = s.user_id]]> - - - ($2, 100)]) - : : : +- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : : +- LogicalProject(user_id=[$1], order_id=[$0], product=[$2]) - : : +- LogicalFilter(condition=[IS NOT NULL($2)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalProject(user_id=[$2], price=[$1]) - : +- LogicalFilter(condition=[>($1, 50)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) - +- LogicalProject(user_id=[$1], location=[$0]) - +- LogicalFilter(condition=[IS NOT NULL($0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Shipments]]) -]]> - (cash, 100)]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, cash], metadata=[]]], fields=[user_id, cash]) + : +- ChangelogNormalize(key=[user_id], condition=[>(cash, 100)]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) : +- Calc(select=[user_id, order_id, product], where=[IS NOT NULL(product)]) : +- TableSourceScan(table=[[default_catalog, default_database, Orders, filter=[]]], fields=[order_id, user_id, product]) @@ -1282,139 +520,40 @@ Calc(select=[user_id, order_id, product, price, location]) - - 100 - ) AS p - ON o.user_id = p.user_id -) AS op -ON u.user_id = op.user_id]]> - - - ($1, 100)]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (price, 100)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) -]]> - - - - - (b, 100)]) - +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) - -advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). -advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. - -related rel plan: -Calc(select=[a, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) -+- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA,D]) - - ]]> - - = 0 -JOIN AddressPK a - ON u.user_id = a.user_id - AND a.location IS NOT NULL]]> - - - =($9, 0))], joinType=[inner]) - : :- LogicalJoin(condition=[AND(=($0, $5), IS NOT NULL($6))], joinType=[inner]) - : : :- LogicalTableScan(table=[[default_catalog, default_database, UsersPK]]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, OrdersPK]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, PaymentsPK]]) - +- LogicalTableScan(table=[[default_catalog, default_database, AddressPK]]) -]]> - =(price, 0)]) - : +- Exchange(distribution=[hash[payment_id, user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) + : +- ChangelogNormalize(key=[payment_id, user_id], condition=[>=(price, 0)]) + : +- Exchange(distribution=[hash[payment_id, user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, PaymentsPK, filter=[]]], fields=[payment_id, user_id, price]) +- Exchange(distribution=[hash[user_id]]) +- ChangelogNormalize(key=[user_id], condition=[IS NOT NULL(location)]) +- Exchange(distribution=[hash[user_id]]) @@ -1422,113 +561,54 @@ Sink(table=[default_catalog.default_database.sink_four_way], fields=[user_id, or ]]> - - - - - - - + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -1777,295 +717,128 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[INNER, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -2081,102 +854,53 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, name, order_id, payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, LEFT], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,name,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[payment_id, user_id], metadata=[]]], fields=[payment_id, user_id]) + +- TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - 10]]> - - - ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - 10)]) + +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> @@ -2194,119 +918,56 @@ LogicalProject(user_id=[$0], name=[$1], order_id=[$3], payment_id=[$6]) == Optimized Physical Plan == Calc(select=[user_id, CAST(_UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS name, order_id, CAST(payment_id AS VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[=(user_id, user_id0), =(user_id, user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id]) - : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- ChangelogNormalize(key=[user_id], condition=[=(name, _UTF-16LE'Gus':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, user_id], where=[>(price, 10)]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) == Optimized Execution Plan == Calc(select=[user_id, CAST('Gus' AS VARCHAR(2147483647)) AS name, order_id, CAST(payment_id AS VARCHAR(2147483647)) AS payment_id]) -+- MultiJoin(commonJoinKey=[user_id], joinTypes=[LEFT, INNER], inputUniqueKeys=[(user_id), (order_id), (payment_id)], joinConditions=[(user_id = user_id0), (user_id = user_id1)], select=[user_id,order_id,user_id0,payment_id,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) payment_id, VARCHAR(2147483647) user_id1)]) ++- MultiJoin(joinFilter=[=($0, $8)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id,name,cash,order_id,user_id0,product,payment_id,price,user_id1], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) product, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id1)]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id]) - : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) - : +- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[], project=[user_id, name], metadata=[]]], fields=[user_id, name]) + : +- ChangelogNormalize(key=[user_id], condition=[(name = 'Gus')]) + : +- Exchange(distribution=[hash[user_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Users, filter=[]]], fields=[user_id, name, cash]) :- Exchange(distribution=[hash[user_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id], metadata=[]]], fields=[order_id, user_id]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, user_id, product]) +- Exchange(distribution=[hash[user_id]]) - +- Calc(select=[payment_id, user_id], where=[(price > 10)]) + +- Calc(select=[payment_id, price, user_id], where=[(price > 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - 10]]> - - - ($7, 10))]) - +- LogicalJoin(condition=[=($0, $8)], joinType=[left]) - :- LogicalJoin(condition=[=($0, $4)], joinType=[left]) - : :- LogicalTableScan(table=[[default_catalog, default_database, Users]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Payments]]) -]]> - (price, 10)]) + +- Calc(select=[payment_id, price, user_id], where=[>(price, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, filter=[]]], fields=[payment_id, price, user_id]) ]]> - - - - - - - - - - - - - - - - - - - - - - - - - - 100]]> - - - (-($1, $2), 100)]) - +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], promo_id=[$7], product_key0=[$8], promo_text=[$9]) - +- LogicalJoin(condition=[=($6, $8)], joinType=[left]) - :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$3], product_key=[$4], quantity=[$5], $f6=[-($1, $2)]) - : +- LogicalProject(product_id=[$0], price=[$1], discount=[$2], sale_id=[$4], product_key=[$5], quantity=[$6]) - : +- LogicalJoin(condition=[=($3, $5)], joinType=[left]) - : :- LogicalProject(product_id=[$0], price=[$1], discount=[$2], $f3=[-($1, $2)]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, Products]]) - : +- LogicalTableScan(table=[[default_catalog, default_database, Sales]]) - +- LogicalTableScan(table=[[default_catalog, default_database, Promotions]]) -]]> - (-(price, discount), 100)]) : : +- Exchange(distribution=[hash[product_id]]) : : +- TableSourceScan(table=[[default_catalog, default_database, Products, filter=[]]], fields=[product_id, price, discount]) : +- Exchange(distribution=[hash[product_key]]) - : +- Calc(select=[product_key, quantity]) - : +- ChangelogNormalize(key=[sale_id]) - : +- Exchange(distribution=[hash[sale_id]]) - : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) + : +- ChangelogNormalize(key=[sale_id]) + : +- Exchange(distribution=[hash[sale_id]]) + : +- TableSourceScan(table=[[default_catalog, default_database, Sales]], fields=[sale_id, product_key, quantity]) +- Exchange(distribution=[hash[product_key]]) - +- Calc(select=[product_key, promo_text]) - +- ChangelogNormalize(key=[promo_id]) - +- Exchange(distribution=[hash[promo_id]]) - +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) + +- ChangelogNormalize(key=[promo_id]) + +- Exchange(distribution=[hash[promo_id]]) + +- TableSourceScan(table=[[default_catalog, default_database, Promotions]], fields=[promo_id, product_key, promo_text]) ]]> - - - - - - + + + + + (b, 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[b, c, d], metadata=[]]], fields=[b, c, d]) + +advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). +advice[2]: [WARNING] The column(s): day(generated by non-deterministic function: CURRENT_TIMESTAMP ) can not satisfy the determinism requirement for correctly processing update message('UB'/'UA'/'D' in changelogMode, not 'I' only), this usually happens when input node has no upsertKey(upsertKeys=[{}]) or current node outputs non-deterministic update messages. Please consider removing these non-deterministic columns or making them deterministic by using deterministic functions. + +related rel plan: +Calc(select=[a, b, DATE_FORMAT(CURRENT_TIMESTAMP(), _UTF-16LE'yyMMdd') AS day], changelogMode=[I,UB,UA,D]) ++- TableSourceScan(table=[[default_catalog, default_database, src1, project=[a, b], metadata=[]]], fields=[a, b], changelogMode=[I,UB,UA,D]) + + ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json index 8a415d1da8c1e..3c6ebe65a68e6 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/plan/four-way-complex-updating-join-with-restore.json @@ -1,7 +1,7 @@ { "flinkVersion" : "2.1", "nodes" : [ { - "id" : 19, + "id" : 25, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -30,7 +30,7 @@ "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[name, user_id_0, cash])" }, { - "id" : 20, + "id" : 26, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -43,7 +43,7 @@ "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", "description" : "Exchange(distribution=[hash[user_id_0]])" }, { - "id" : 21, + "id" : 27, "type" : "stream-exec-changelog-normalize_1", "configuration" : { "table.exec.mini-batch.enabled" : "false", @@ -66,7 +66,7 @@ "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", "description" : "ChangelogNormalize(key=[user_id_0])" }, { - "id" : 22, + "id" : 28, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -79,7 +79,7 @@ "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", "description" : "Exchange(distribution=[hash[user_id_0]])" }, { - "id" : 23, + "id" : 29, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -103,25 +103,113 @@ } } } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 0 ], [ 2 ] ], + "producedType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, product, user_id_1])" + "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])" }, { - "id" : 24, + "id" : 30, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 2 ] + "keys" : [ 1 ] }, "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", + "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id_1]])" }, { - "id" : 25, + "id" : 31, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 1 ], + "rightKeys" : [ 1 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null + }, + "leftUpsertKeys" : [ [ 1 ] ], + "rightUpsertKeys" : [ [ 0 ] ], + "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<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(user_id_0 = user_id_1)], select=[name, user_id_0, cash, order_id, user_id_1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 32, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647)>", + "description" : "Calc(select=[name, user_id_0, cash, order_id])" + }, { + "id" : 33, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[user_id_0]])" + }, { + "id" : 34, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -150,7 +238,7 @@ "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[user_id_2, payment_id, price])" }, { - "id" : 26, + "id" : 35, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -163,149 +251,14 @@ "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", "description" : "Exchange(distribution=[hash[user_id_2]])" }, { - "id" : 27, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`Shipments`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "location", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "user_id_3", - "dataType" : "VARCHAR(2147483647)" - } ] - } - } - } - }, - "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])" - }, { - "id" : 28, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 1 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", - "description" : "Exchange(distribution=[hash[user_id_3]])" - }, { - "id" : 29, - "type" : "stream-exec-multi-join_1", - "joinTypes" : [ "INNER", "LEFT", "INNER", "LEFT" ], - "joinConditions" : [ { - "kind" : "LITERAL", - "value" : true, - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 5, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$AND$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 6, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$OR$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 8, - "type" : "INT" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$<$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 8, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 6, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 10, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "multiJoinCondition" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$AND$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 6, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { + "id" : 36, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 1 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : { "kind" : "CALL", "syntax" : "BINARY", "internalName" : "$OR$1", @@ -319,7 +272,7 @@ "type" : "INT" }, { "kind" : "INPUT_REF", - "inputIndex" : 8, + "inputIndex" : 6, "type" : "INT" } ], "type" : "BOOLEAN" @@ -329,7 +282,7 @@ "internalName" : "$<$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 8, + "inputIndex" : 6, "type" : "INT" }, { "kind" : "LITERAL", @@ -339,80 +292,150 @@ "type" : "BOOLEAN" } ], "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "joinAttributeMap" : { - "1" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 1, - "rightInputId" : 1, - "rightFieldIndex" : 2 - } ], - "2" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 1, - "rightInputId" : 2, - "rightFieldIndex" : 0 - } ], - "3" : [ { - "leftInputId" : 2, - "leftFieldIndex" : 0, - "rightInputId" : 3, - "rightFieldIndex" : 1 - } ] + } }, - "inputUniqueKeys" : [ [ [ 1 ] ], [ [ 0 ] ], [ [ 1 ] ], [ ] ], + "rightUpsertKeys" : [ [ 1 ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", - "name" : "input-state-0" + "name" : "leftState" }, { "index" : 1, "ttl" : "0 ms", - "name" : "input-state-1" + "name" : "rightState" + } ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 }, { - "index" : 2, - "ttl" : "0 ms", - "name" : "input-state-2" + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", + "description" : "Join(joinType=[InnerJoin], where=[((user_id_0 = user_id_2) AND ((cash >= price) OR (price < 0)))], select=[name, user_id_0, cash, order_id, user_id_2, payment_id, price], leftInputSpec=[NoUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 37, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" }, { - "index" : 3, - "ttl" : "0 ms", - "name" : "input-state-3" + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", + "description" : "Calc(select=[name, user_id_0, order_id, user_id_2, payment_id])" + }, { + "id" : 38, + "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 1 ] + "keys" : [ 3 ] }, "damBehavior" : "PIPELINED", "priority" : 0 - }, { + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", + "description" : "Exchange(distribution=[hash[user_id_2]])" + }, { + "id" : 39, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`Shipments`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "location", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "user_id_3", + "dataType" : "VARCHAR(2147483647)" + } ] + } + } + } + }, + "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])" + }, { + "id" : 40, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 2 ] + "keys" : [ 1 ] }, "damBehavior" : "PIPELINED", - "priority" : 1 + "priority" : 0 + } ], + "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[user_id_3]])" + }, { + "id" : 41, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 3 ], + "rightKeys" : [ 1 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null + }, + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "leftState" }, { + "index" : 1, + "ttl" : "0 ms", + "name" : "rightState" + } ], + "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 2 + "priority" : 0 }, { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 1 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 3 + "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT, `location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", - "description" : "MultiJoin(joinFilter=[AND(=($1, $6), OR(>=($2, $8), <($8, 0)))], joinTypes=[[INNER, LEFT, INNER, LEFT]], joinConditions=[[true, =($1, $5), AND(=($1, $6), OR(>=($2, $8), <($8, 0))), =($6, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:1;RightInputId:1;RightFieldIndex:2;], 2=[LeftInputId:0;LeftFieldIndex:1;RightInputId:2;RightFieldIndex:0;], 3=[LeftInputId:2;LeftFieldIndex:0;RightInputId:3;RightFieldIndex:1;]}], select=[name,user_id_0,cash,order_id,product,user_id_1,user_id_2,payment_id,price,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) name, VARCHAR(2147483647) user_id_0, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(user_id_2 = user_id_3)], select=[name, user_id_0, order_id, user_id_2, payment_id, location, user_id_3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" }, { - "id" : 30, + "id" : 42, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -424,15 +447,15 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 2, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 7, + "inputIndex" : 4, "type" : "VARCHAR(2147483647) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 9, + "inputIndex" : 5, "type" : "VARCHAR(2147483647)" } ], "condition" : null, @@ -446,7 +469,7 @@ "outputType" : "ROW<`user_id_0` VARCHAR(2147483647) NOT NULL, `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `location` VARCHAR(2147483647)>", "description" : "Calc(select=[user_id_0, name, order_id, payment_id, location])" }, { - "id" : 31, + "id" : 43, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -492,88 +515,130 @@ "description" : "Sink(table=[default_catalog.default_database.sink], fields=[user_id_0, name, order_id, payment_id, location])" } ], "edges" : [ { - "source" : 19, - "target" : 20, + "source" : 25, + "target" : 26, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 20, - "target" : 21, + "source" : 26, + "target" : 27, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 21, - "target" : 22, + "source" : 27, + "target" : 28, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 29, + "target" : 30, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 25, - "target" : 26, + "source" : 28, + "target" : 31, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 27, - "target" : 28, + "source" : 30, + "target" : 31, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 22, - "target" : 29, + "source" : 31, + "target" : 32, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 24, - "target" : 29, + "source" : 32, + "target" : 33, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 26, - "target" : 29, + "source" : 34, + "target" : 35, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 28, - "target" : 29, + "source" : 33, + "target" : 36, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 29, - "target" : 30, + "source" : 35, + "target" : 36, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 30, - "target" : 31, + "source" : 36, + "target" : 37, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 39, + "target" : 40, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 40, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 41, + "target" : 42, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 42, + "target" : 43, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-complex-updating-join-with-restore/savepoint/_metadata index 061d0b08163df013497e898e81c8c18037fa96da..cd2f1cf00c976a70addd4b0750148759c5a6bc72 100644 GIT binary patch delta 3127 zcmc&$d2AI$9Nt~7BBiwODy@P&g|;$%bL~|v6%a!!6cA7otYvn0wmf{j-g~W&^Fl z?|b1P?Y_sg?Nh7smArW@>8_->O&LdVyDJ;&j59pAV)49B?m76uvJai_cg%^2g)<8) z^IEzST`MUHGg12**|3V)}Kiq{^{Y54z9gLW(Ik4W2!c}H~6V;NY$oP$oFjd z_QXp7uYb@_BzZ%;pQYk{chcjjbbPH5xBP)r|7s%*`}>Ink%Q>H{cdM(3d*OWC%!zs zrmriJ0n|X2q1zBaN}-9dW$N)X>*qakBN@QCYgr~Bh%;hao{mf$hc=~-jxgYs4FUB^ zd>vQ}A%Hk%hNDBOBS%qU-NB1D&sW8&HD68Al>8+3Ga$v??v$4n+u4snL211v4#j)L z_SSL1s)7aTpFYH@f@89GU9h+88&#brZk$s*Z4K*Z^_hVl-j_)v)Agyou1unn*Dvqu zNpi8YdHN_#nGq(IWfQ}2wi7F{+lLiJQDxDTR+)31C^u?ETk1%`#8jeV^~F0oD;6v| z7JR&Lak%w$!PE`g)dx3K&1>!MWS*jIC2|h=Y?SBVDWVLGwDV*T%F@FII(b(n43C#3 zT2qbc2BB^$Rm;Vq%}~W+aC50zB93gS(QX|OsIf$Rv30z-t+5b96GLSPg`?#{f0;N8 z|1q&RxOGN2ibSK(e;gHU2MgF3Zeh~4Sz@4Z@&da}8ISGuba!9MdDQ7O70PwHNK8Tb8`nym-`}r8jLkbkR|{?io~Pat10ZSEn}hAz@HQlv>n< zm*c3&bRBGQ0+C1rd)~5c6T-F>fp)l4$#$| z>SW2pTK@8`Q~PV4({^?~FlzTTKi@w%B00f{x?iGbt2X}G-H_1T zkgU5&oq6w6*{-$IPO^KZ?|I{w)gy=33MXatws3z|ZwrSnhu-Qo^RQ={01~bPs+XYf zj-d{G31iE*OcPns-4vbpH@&@Ddj3K@a9@k(^w$~wKP^wki63(k*Uji zk#N`}TW5(=TdKswq1yV3wKaG565Zp7M7MOdvAq+Xo^blJlJ9oz>3H<(-CpkQYYLVn zy6$c$Ps@5GB{H9q_ONmK26bcc^6Q=Z_4e~`?>o7F$DD<+P9;C=k`6K5m&~N)hLkZo zo-AA68Gb2B#j~n%iEXYlIgHQ5lHwdY5($iAV7WE}wrQIj=@#*!{Nt@SF1I&!#~ssx zy+z+l3LY-Ph2mW0HKJCQ_bVLnW#t4xD~rW&<+NOEk$AmI7cW&cXo?cxZNu7cebc~L delta 3340 zcmdT`TWl0n7@nEZ3%#+W+slf9<DCA~2?j?FUlL>)I0o}ZHPQS;8f-v{Ug#K<^&kIcG^%Ht{LSGoy z^Gig5l1Qgl7wSdv5b5f4y?7#|kx;xd>gq8Z(|2f!gOj#xMq_=^WE?Qr(j8hC+SU_` zBxMBx2rRHpF?3*RrfzGBs)rJ&*X!&ZH{ zJlS?kAOlm^TmlWvMyeppZ%xgO=t0^#)JiW6meJ7A>?}+0)3rn8)#JqrLLM7{%mdTi zs4wcpk{TOYf{#8qSY4j6PM8#K%1WDfa9Zfwgf~-w*VNxGpw;DZ8Gc%6geYOc|?eE1+%bwn1Er zAcrFRuX%ZmsMG1t0OuqKJM7`z{C&oO9aoyKo|ygNC9*X)QeqK52`XAkgNpdLzNo3b6q7c*|0UFT_>v2ZipbFUw)4v|Lr_NWI^85N7|f z;_6QuS{D3BKl}CQq0xaFG4*r9pVA|5LD4+d(vT+;)6--iH49M z6yHo8SwCM&z2A7TkRA_|G-+(#RBdA4jvp0I#H*s;uRON5aObK$pWd-=*g!T=umx4s zFkKJmIwBSX4m;3>WnlZU?3;y6U?bqlSW#K0Wsc*>wuV%8gN}_9=&2B=knIzjRmdBK z6OCT1{T9xBN|~A3yW()w$*<1e(WvXFwyS7RWhpg<-z=!Bwyhf2bxh~+5LC0U<7vc} z6>KZ4QJVl669i;S_e@(Qx^CKl6I3p0WuJw}YWC{wP}!L*Qtfb*&&b=@RB+ja0P5ZQ z@|sgCy^)WN5l&aB?zhzShSx=@{_E7JxI~neGz^b|{s;3_xSuzZhc#%jmjL@75Il?V z76REeJS+oj>WKA^*c(F-mhDb6xNMp*^LAr$B-Y)NlmU(9UQ3O&o)lC0i^j4`|Lvi0 z4|*Qu(Sse;`(wHEhn@b?q~L4WhI&z`CmHDswL}uh#Z;{Mo!)%un*#BLX`CF1xM0xX z{Hp!2=4@Y!FfB<(chr@1CxxP(gclD-+^`bf?)6czpgH)wuuG7{U0h`+h1}35` zZ|lEx>(+~Wu9&{lA^+Ejm(?F#M z=h2m{Nx)Jc8d&Q86RfFNr9CvDv6sItrKkKNH0v+?JM3Ek= zs1EXftx9%rUBj-7@Si;<1GJJTIfBH7?L0y4GCs>C6Uc)n4LogBP^7ClLKgGD>EI?F z$R!gTWaCML)k+|?(Pv12o~@Yf%eZHXq$9eYb`d#K81oA(W2w@Tan2N$c0Oa2(zBUj z+Iix81PoLzV35TE0U8gaAv;B?a`^vNTOaKTRMIahCn@VX@zeY*ymB5mrOSpb|5VQW c?otTPcg9#X4q-b_dJpHf0yQcBn>VDt0mh!M(f|Me diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json index 4b020388a6177..72047cabe9227 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/plan/four-way-join-no-common-join-key-with-restore.json @@ -1,7 +1,7 @@ { "flinkVersion" : "2.1", "nodes" : [ { - "id" : 32, + "id" : 44, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -25,12 +25,21 @@ } } } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 0 ], [ 1 ] ], + "producedType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] }, - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[name, user_id_0, cash])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, Users, project=[name, user_id_0], metadata=[]]], fields=[name, user_id_0])" }, { - "id" : 33, + "id" : 45, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -40,10 +49,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL>", "description" : "Exchange(distribution=[hash[user_id_0]])" }, { - "id" : 34, + "id" : 46, "type" : "stream-exec-changelog-normalize_1", "configuration" : { "table.exec.mini-batch.enabled" : "false", @@ -63,10 +72,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL>", "description" : "ChangelogNormalize(key=[user_id_0])" }, { - "id" : 35, + "id" : 47, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -76,10 +85,10 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL>", "description" : "Exchange(distribution=[hash[user_id_0]])" }, { - "id" : 36, + "id" : 48, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -103,25 +112,109 @@ } } } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 0 ], [ 2 ] ], + "producedType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)> NOT NULL" + } ] }, - "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[order_id, product, user_id_1])" + "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, Orders, project=[order_id, user_id_1], metadata=[]]], fields=[order_id, user_id_1])" }, { - "id" : 37, + "id" : 49, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 2 ] + "keys" : [ 1 ] }, "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", + "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id_1]])" }, { - "id" : 38, + "id" : 50, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 1 ], + "rightKeys" : [ 1 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null + }, + "leftUpsertKeys" : [ [ 1 ] ], + "rightUpsertKeys" : [ [ 0 ] ], + "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<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(user_id_0 = user_id_1)], select=[name, user_id_0, order_id, user_id_1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 51, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647)>", + "description" : "Calc(select=[name, user_id_0, order_id])" + }, { + "id" : 52, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[user_id_0]])" + }, { + "id" : 53, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -145,12 +238,21 @@ } } } - } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 0 ], [ 1 ] ], + "producedType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] }, - "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[user_id_2, payment_id, price])" + "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments, project=[user_id_2, payment_id], metadata=[]]], fields=[user_id_2, payment_id])" }, { - "id" : 39, + "id" : 54, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -160,127 +262,88 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", + "outputType" : "ROW<`user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", "description" : "Exchange(distribution=[hash[user_id_2]])" }, { - "id" : 40, - "type" : "stream-exec-multi-join_1", - "joinTypes" : [ "INNER", "LEFT", "INNER" ], - "joinConditions" : [ { - "kind" : "LITERAL", - "value" : true, - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 5, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 6, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "multiJoinCondition" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 6, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" + "id" : 55, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 1 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null }, - "joinAttributeMap" : { - "1" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 1, - "rightInputId" : 1, - "rightFieldIndex" : 2 - } ], - "2" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 1, - "rightInputId" : 2, - "rightFieldIndex" : 0 - } ] - }, - "inputUniqueKeys" : [ [ [ 1 ] ], [ [ 0 ] ], [ [ 1 ] ] ], + "rightUpsertKeys" : [ [ 1 ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", - "name" : "input-state-0" + "name" : "leftState" }, { "index" : 1, "ttl" : "0 ms", - "name" : "input-state-1" - }, { - "index" : 2, - "ttl" : "0 ms", - "name" : "input-state-2" + "name" : "rightState" } ], "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 1 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", "priority" : 0 }, { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 2 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 1 + "priority" : 0 + } ], + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", + "description" : "Join(joinType=[InnerJoin], where=[(user_id_0 = user_id_2)], select=[name, user_id_0, order_id, user_id_2, payment_id], leftInputSpec=[NoUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 56, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 2 + "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", - "description" : "MultiJoin(joinFilter=[=($1, $6)], joinTypes=[[INNER, LEFT, INNER]], joinConditions=[[true, =($1, $5), =($1, $6)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:1;RightInputId:1;RightFieldIndex:2;], 2=[LeftInputId:0;LeftFieldIndex:1;RightInputId:2;RightFieldIndex:0;]}], select=[name,user_id_0,cash,order_id,product,user_id_1,user_id_2,payment_id,price], rowType=[RecordType(VARCHAR(2147483647) name, VARCHAR(2147483647) user_id_0, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) payment_id, INTEGER price)])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", + "description" : "Calc(select=[name, user_id_0, order_id, payment_id])" }, { - "id" : 41, + "id" : 57, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", - "keys" : [ 7 ] + "keys" : [ 3 ] }, "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT>", + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL>", "description" : "Exchange(distribution=[hash[payment_id]])" }, { - "id" : 42, + "id" : 58, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -301,7 +364,7 @@ "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Shipments]], fields=[location, user_id_3])" }, { - "id" : 43, + "id" : 59, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -314,65 +377,41 @@ "outputType" : "ROW<`location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id_3]])" }, { - "id" : 44, - "type" : "stream-exec-multi-join_1", - "joinTypes" : [ "INNER", "LEFT" ], - "joinConditions" : [ { - "kind" : "LITERAL", - "value" : true, - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 7, - "type" : "VARCHAR(2147483647) NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 10, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "joinAttributeMap" : { - "1" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 7, - "rightInputId" : 1, - "rightFieldIndex" : 1 - } ] + "id" : 60, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 3 ], + "rightKeys" : [ 1 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null }, - "inputUniqueKeys" : [ [ ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", - "name" : "input-state-0" + "name" : "leftState" }, { "index" : 1, "ttl" : "0 ms", - "name" : "input-state-1" + "name" : "rightState" } ], "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 7 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", "priority" : 0 }, { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 1 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 1 + "priority" : 0 } ], - "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `cash` INT, `order_id` VARCHAR(2147483647), `product` VARCHAR(2147483647), `user_id_1` VARCHAR(2147483647), `user_id_2` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT, `location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", - "description" : "MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT]], joinConditions=[[true, =($7, $10)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:7;RightInputId:1;RightFieldIndex:1;]}], select=[name,user_id_0,cash,order_id,product,user_id_1,user_id_2,payment_id,price,location,user_id_3], rowType=[RecordType(VARCHAR(2147483647) name, VARCHAR(2147483647) user_id_0, INTEGER cash, VARCHAR(2147483647) order_id, VARCHAR(2147483647) product, VARCHAR(2147483647) user_id_1, VARCHAR(2147483647) user_id_2, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) location, VARCHAR(2147483647) user_id_3)])" + "outputType" : "ROW<`name` VARCHAR(2147483647), `user_id_0` VARCHAR(2147483647) NOT NULL, `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `location` VARCHAR(2147483647), `user_id_3` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(payment_id = user_id_3)], select=[name, user_id_0, order_id, payment_id, location, user_id_3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" }, { - "id" : 45, + "id" : 61, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -384,15 +423,15 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 2, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 7, + "inputIndex" : 3, "type" : "VARCHAR(2147483647) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 9, + "inputIndex" : 4, "type" : "VARCHAR(2147483647)" } ], "condition" : null, @@ -406,7 +445,7 @@ "outputType" : "ROW<`user_id_0` VARCHAR(2147483647) NOT NULL, `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647) NOT NULL, `location` VARCHAR(2147483647)>", "description" : "Calc(select=[user_id_0, name, order_id, payment_id, location])" }, { - "id" : 46, + "id" : 62, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -452,102 +491,130 @@ "description" : "Sink(table=[default_catalog.default_database.sink], fields=[user_id_0, name, order_id, payment_id, location])" } ], "edges" : [ { - "source" : 32, - "target" : 33, + "source" : 44, + "target" : 45, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 33, - "target" : 34, + "source" : 45, + "target" : 46, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 34, - "target" : 35, + "source" : 46, + "target" : 47, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 36, - "target" : 37, + "source" : 48, + "target" : 49, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 38, - "target" : 39, + "source" : 47, + "target" : 50, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 35, - "target" : 40, + "source" : 49, + "target" : 50, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 37, - "target" : 40, + "source" : 50, + "target" : 51, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 39, - "target" : 40, + "source" : 51, + "target" : 52, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 40, - "target" : 41, + "source" : 53, + "target" : 54, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 42, - "target" : 43, + "source" : 52, + "target" : 55, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 41, - "target" : 44, + "source" : 54, + "target" : 55, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 43, - "target" : 44, + "source" : 55, + "target" : 56, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 44, - "target" : 45, + "source" : 56, + "target" : 57, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 45, - "target" : 46, + "source" : 58, + "target" : 59, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 57, + "target" : 60, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 59, + "target" : 60, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 60, + "target" : 61, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 61, + "target" : 62, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/four-way-join-no-common-join-key-with-restore/savepoint/_metadata index f53211043d60e0457995ea123f9ad15aadae811a..6b5de7c2a7d8426c4572bc346bfb60988c5587c5 100644 GIT binary patch delta 3074 zcmc(hYit!o6vuaO(-*I{mjXo#v<+{V-kIH*-PwY~fY^dSQ63U1DYLt??QL%#+6*VTv$K9FF++1?gPBu1lwNp@#v z&&*`bp7a0hdHwCc{;h$XV^$ASg3B73q7E-9D37{48mqu`jgpCJWvuUYqWw^MQR%|c zaT{v(cAk5yQ&ALEUTvyUVC&Ro9yB!#o0wq}xh6B1g|byOhL(qFn(D)GU>XL(M5DTeuzaKj zBB2;-=8X;E`Z&O@#V`dL)GP=Yu^H1t4%2Pw5*;w?5`cNGrvV!}I`D|*5qUQ1GEZ#U z-Ybsp2reMH^pxoRp1x7I>cWY%qo+>1{O!YC`=)je@{}1GB9tS?fQGHP&@n9LS{OQr zL5(_0^w3%N+EtI57DS#4kO_5wQLb(Rn>rf64)S!v<(lPS>FSNbh4OP%5VA%|C#14Q z4!$_}?$pWWF0St`i;kLkZ0H=ZZ^zig?7A#9kaaWBscumN*{{fcj_faWZi20%3f z*k4etiX|)0CC;w+R8><%XK9zXJY=m{ytRA8@MyTI`i=#OUi)UMS|WbS8LJiu7|s(* z+B33b2BgYMCcK~W(|xNL9WD~nYsRS4#gFY{#n0`zBD;1{prl2>V4hsMIG86kc4Q2{ zy;M<3L@4NQ>)Sr-`0KXPD#~k%#CMVWsUwz!>uXC2V)9;LXfkAmR_PDE9+IB>{O6%j zrBhB(xv2m^t@_TU&SL0RZAM8bu1Fzz$>t^Mj4HbN@u_X z>IZDQN4-~(tN(g(VXAm(`VVd4TWUzOsec6SwXC^LJ%hUlxRPT&!6*clq$xlI*_!2Y zDQ-KFoyvw6Wz1s{r-C3~*r?KxgOm@?WNf{1qvzsYx9g`Kb*Xe^qIhmcT0W8PrxWSE zoV4hOnpw2?dh1uyn#Skli(`G0R&UEH!`KHu?#Cgyk`p?0a>4p|0=8o!fS~1g9!1pF z-9a>j{`Y`=G(5jYj9vNW+Pz1`C!<@UU2|{6KQLlrl7?B?Ng8H3_l1Tgw_L1A(Kt52 zl7^=30n0EUpoAJ8mVZy|*@}`q|L!z=PButTS3zW;3Ocv``V&1l<=^aCTKRrT_xD|c#Ncw%^lVvPK!mX8nwCW% z@oWR?4s&hiepErWga|{f4Wu?5fN{{VfQ4)bu#K5xb6Ik|*1BCy76yhd1tTeiM?bQ1 zu9kk#n|Nx%ekJY{-IhZ3$4M#V*hwkm9Jnu1(7DbCm2{-82Bj3B=>iLK2Y8fH?sA(z zO%?{hop+Z)Pa?gkJMBMcf&61FD7jDe_Gd{zoO))(-3yOviA#@64hUG_OXY%^Ek%&T zY_jASC`b;qLsjz&4mD2{b<~d-G)+An}69*q}?}cARb? z$g$I=sSUbl|T z09<&NIN}rlWWaA~oSj|QcCaezwau^Q1ED266k1TCfKndms6Qv~2>^UY$E-P}`8Sdm znH58WmJ5KtRIc%RLg&dPf1B4FLi~cz9R5sbL9jM=;i==YuO>OhOtJU@;F0OBy`{I(;d18Vj`nUv|42&I2nTH6` zTuWE5jx?-bVyjfe2B!S$f%Y?=%AO&Pg-jJ{riVo;GsA?oXE_i#y6M=|b&%@u=I{}I z|3K|BN`$YJ6nri1TlDt6ny$Ed@Z;ZBz8v=bEc&WYRs7SA>RJd>q%op0>|)PR728m; zvSqmA%zLU^njsq06-N=Cx;BB9suQTIoOhRCyAs}|4MqWiK{bZ(8;@%H*kJyJ_Eewt)IZVQMfdK$`MS~mcc1_1x7Y7m`0S&{XfL4WX)ka%PkRmC zS)x6^hI1{=fjTv?pgqBKqHI%jQQ=vZ>R5`Zdo}=#!-Do&Lh=l{unj? zofmT-mFI8YS~9&%|^ZokE>!MLQqpIXfSV z4Vkra8Q_~US62RMht+u~bmk`>+v4a{A2?;3TKTUlt9uCc>pEPhb@m)K@2l@g##h#E zd-A)TV|R!pinWewc(%AUI7p+Wjy0w#BH>iaV4m&j;wErb>N=!r6x$wy80knd{)aUV# zT+>4gKO#mUBI}QAzjkWk{D-xtURyA_c&=rtsv@dfdYfhLk5Gi4kH&@wx>YiI^t5oR7MbQ3QraSBC7gws^tPQxqIegzYl!$ W$(LVTwijI5TO$H8Hw!oYDE|Q_t~HVX diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/plan/three-way-inner-join-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/plan/three-way-inner-join-with-restore.json index 3b650218933dc..155df3b652215 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/plan/three-way-inner-join-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/plan/three-way-inner-join-with-restore.json @@ -70,6 +70,79 @@ "description" : "Exchange(distribution=[hash[user_id]])" }, { "id" : 5, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null + }, + "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<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Join(joinType=[InnerJoin], where=[(user_id = user_id0)], select=[user_id, name, user_id0, order_id], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" + }, { + "id" : 6, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Calc(select=[user_id, name, order_id])" + }, { + "id" : 7, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[user_id]])" + }, { + "id" : 8, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -90,7 +163,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[user_id, payment_id])" }, { - "id" : 6, + "id" : 9, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -103,131 +176,41 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id]])" }, { - "id" : 7, - "type" : "stream-exec-multi-join_1", - "joinTypes" : [ "INNER", "INNER", "INNER" ], - "joinConditions" : [ { - "kind" : "LITERAL", - "value" : true, - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "multiJoinCondition" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$AND$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "joinAttributeMap" : { - "1" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 0, - "rightInputId" : 1, - "rightFieldIndex" : 0 - } ], - "2" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 0, - "rightInputId" : 2, - "rightFieldIndex" : 0 - } ] + "id" : 10, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null }, - "inputUniqueKeys" : [ [ ], [ ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", - "name" : "input-state-0" + "name" : "leftState" }, { "index" : 1, "ttl" : "0 ms", - "name" : "input-state-1" - }, { - "index" : 2, - "ttl" : "0 ms", - "name" : "input-state-2" + "name" : "rightState" } ], "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", "priority" : 0 }, { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 1 - }, { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 2 + "priority" : 0 } ], - "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `user_id1` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", - "description" : "MultiJoin(joinFilter=[AND(=($0, $4), =($0, $2))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,user_id0,order_id,user_id1,payment_id], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) payment_id)])" + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", + "description" : "Join(joinType=[InnerJoin], where=[(user_id = user_id0)], select=[user_id, name, order_id, user_id0, payment_id], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" }, { - "id" : 8, + "id" : 11, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -239,11 +222,11 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 2, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 5, + "inputIndex" : 4, "type" : "VARCHAR(2147483647)" } ], "condition" : null, @@ -257,7 +240,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "Calc(select=[user_id, name, order_id, payment_id])" }, { - "id" : 9, + "id" : 12, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -314,22 +297,22 @@ }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 2, + "target" : 5, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 7, + "source" : 4, + "target" : 5, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 7, + "source" : 5, + "target" : 6, "shuffle" : { "type" : "FORWARD" }, @@ -341,19 +324,40 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" }, { "source" : 7, - "target" : 8, + "target" : 10, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-inner-join-with-restore/savepoint/_metadata index 377ec1862c7975e15fba6bcea8320f0f7466db77..66aa54c2fbecedeee8c070ff4d0a8305ef9f991e 100644 GIT binary patch delta 1418 zcmchXPiRy}9LL|R>AG68OHe7XMm7mvDwCa=H#2W$5v7Q+gd_yxrQN2>%zN{6{j<^C zWYK^$z4RibtrP6Y_R=Vdi1<9%L#3BO|IliiDtc1!su0A3h!W=|2p;s94*UjwGe3sk z@ALc4Z|)3ua2kB`-Um4^KU%9FC>NVK z&$@3%YoAZ?l8BErTKjktwOYPFSYvv$6&^a$KyK|lT-sCGKUuF$Kq7<`$d_DDL4p7W zjF+0KWe$(m>Jw1$NE-$TMqDt0R47S_U_Vkmi=i<(gw#+5wbmS(KoUckP*ee)Yz5h_ z4SdYY@@l9Kh9h6a_kPaym-_Ea&E)UCzsDuoX<_fXAFmx9pKOGtTpUry4w?Eyvm(a4 zoTxSzkNNJIpSpHl*<3g`x8jShCy#rc2b@g#ihw~zxPUZJ7-|Ozq5wxm`jJ!uDEnLY3p>5($7Rh##2RxQ3TU_vAZ0{B zB`AjyhcV++Vl6?GX3~=zaBZz%lqv{S_dDnDmQVYxUcYt$BzI22f%7xTe9xs!CSOjv zd$)m1{%_mei>%+het8RU?c;<*P)T1y8Vdu39|n+zCgPD#38AZz=M}W;eb!6<;hQpr zjJM{Xi6>yggyW5uF*a?)=01C|!43QL)ib(2cweq}n>q>q* z)s$9W+nXsi+W<)^+RniaYn`nh`pBI)E|w2<*h@nleG3aI*QJ%);U+-^<5YckaV(tO z{^jJlsq(4dUp#v;-+%wrSg_lMXO~rRvBW`$jD|R%1X83W6v|DyP>NW^F^L2P*nOGj JNp<8&<|R&gs$c*B delta 1671 zcmd5+U2IfE6rOv#TPmem+EOT{WJ_sjm0|ClJ2UqVB&C~T+*(vxiNV%py+3ntp?kaR z-UaFp6~c>&8pynmC=X4vPd*s;MIY4oAh9MjA@Wuqnm~BbYE1fI0(G6cP>7)hUz&K5 zIdjgLGv}P|o_prT73HT{<>JR_;@3 z+qYY?NGs#o6Yj~|ul-D|;SWFek~2B;*O^PH7V-69qnJ&6yspKX)bj{25hPSIA?7ZI zj!iVkP1AQcbC}B&L;TROBF^tozK!$u1GS~4e z;+jl16cnL)deifzi7@+|Z049b4mO#YT%^O}((wIT*Z)|&die0%t>3z8=3n_zwxqnW z^CMN=^t(F^Wmt2cY}i z@-O>4)h#uk=>Z0q4f3TBYUSa9?s)UV2f7#@_*2;<^jMc@h&?HS##V8&by+;sJikFX z9gBJ@{%S778xMh4C>GUyPvk>TUn=rKChKKR@F}6!5B8h{XMw7mjkb6QHoTk~8Azu_ zrYn`onaK9|r%NDSEP%DssMR#;KhucmmZN=Zhb%@RqcZ9y3tqL3s-;DV?5z)yihey? z-q`b%B2e9{<%R<^wg!ImKgQqx2>cuK_Ee4sq3pze{Q_cfC??)ar<73vX~EDC zCEs>$^~hbEOSg!HbbBpOrRG$f(^9I^s&hhW_nn969#N?V{+71&_E#MxTlI~CN4~41 z-xbb@vB8ejB+", "description" : "TableSourceScan(table=[[default_catalog, default_database, UsersWithProctime]], fields=[user_id_0, name])" }, { - "id" : 48, + "id" : 64, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -81,7 +81,7 @@ "outputType" : "ROW<`user_id_0` VARCHAR(2147483647) NOT NULL, `name` VARCHAR(2147483647), `proctime` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL>", "description" : "Calc(select=[user_id_0, name, PROCTIME_MATERIALIZE(PROCTIME()) AS proctime])" }, { - "id" : 49, + "id" : 65, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -94,7 +94,7 @@ "outputType" : "ROW<`user_id_0` VARCHAR(2147483647) NOT NULL, `name` VARCHAR(2147483647), `proctime` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL>", "description" : "Exchange(distribution=[hash[user_id_0]])" }, { - "id" : 50, + "id" : 66, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -138,7 +138,7 @@ "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, OrdersWithRowtime]], fields=[order_id, user_id_1, rowtime])" }, { - "id" : 51, + "id" : 67, "type" : "stream-exec-watermark-assigner_1", "watermarkExpr" : { "kind" : "INPUT_REF", @@ -172,7 +172,7 @@ }, "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])" }, { - "id" : 52, + "id" : 68, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -208,7 +208,7 @@ "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", "description" : "Calc(select=[order_id, user_id_1, CAST(rowtime AS TIMESTAMP(3)) AS rowtime])" }, { - "id" : 53, + "id" : 69, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -221,7 +221,7 @@ "outputType" : "ROW<`order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>", "description" : "Exchange(distribution=[hash[user_id_1]])" }, { - "id" : 54, + "id" : 70, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -250,7 +250,7 @@ "outputType" : "ROW<`payment_id` VARCHAR(2147483647) NOT NULL, `price` INT, `user_id_2` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[payment_id, price, user_id_2])" }, { - "id" : 55, + "id" : 71, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -263,7 +263,7 @@ "outputType" : "ROW<`payment_id` VARCHAR(2147483647) NOT NULL, `price` INT, `user_id_2` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id_2]])" }, { - "id" : 56, + "id" : 72, "type" : "stream-exec-multi-join_1", "joinTypes" : [ "INNER", "INNER", "INNER" ], "joinConditions" : [ { @@ -313,7 +313,7 @@ "type" : "VARCHAR(2147483647) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 8, + "inputIndex" : 4, "type" : "VARCHAR(2147483647)" } ], "type" : "BOOLEAN" @@ -327,7 +327,7 @@ "type" : "VARCHAR(2147483647) NOT NULL" }, { "kind" : "INPUT_REF", - "inputIndex" : 4, + "inputIndex" : 8, "type" : "VARCHAR(2147483647)" } ], "type" : "BOOLEAN" @@ -335,6 +335,17 @@ "type" : "BOOLEAN" }, "joinAttributeMap" : { + "0" : [ { + "leftInputId" : -1, + "leftFieldIndex" : -1, + "rightInputId" : 0, + "rightFieldIndex" : 0 + }, { + "leftInputId" : -1, + "leftFieldIndex" : -1, + "rightInputId" : 0, + "rightFieldIndex" : 0 + } ], "1" : [ { "leftInputId" : 0, "leftFieldIndex" : 0, @@ -348,7 +359,7 @@ "rightFieldIndex" : 2 } ] }, - "inputUniqueKeys" : [ [ [ 0 ] ], [ [ 0 ] ], [ ] ], + "inputUniqueKeys" : [ [ [ 0 ] ], [ [ 0 ] ], [ [ 0 ] ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", @@ -385,9 +396,9 @@ "priority" : 2 } ], "outputType" : "ROW<`user_id_0` VARCHAR(2147483647) NOT NULL, `name` VARCHAR(2147483647), `proctime` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL, `order_id` VARCHAR(2147483647) NOT NULL, `user_id_1` VARCHAR(2147483647), `rowtime` TIMESTAMP(3), `payment_id` VARCHAR(2147483647) NOT NULL, `price` INT, `user_id_2` VARCHAR(2147483647)>", - "description" : "MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id_0,name,proctime,order_id,user_id_1,rowtime,payment_id,price,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) proctime, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, TIMESTAMP(3) rowtime, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2)])" + "description" : "MultiJoin(joinFilter=[AND(=($0, $8), =($0, $4))], joinTypes=[[INNER, INNER, INNER]], joinConditions=[[true, =($0, $4), =($0, $8)]], joinAttributeMap=[{0=[LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;, LeftInputId:-1;LeftFieldIndex:-1;RightInputId:0;RightFieldIndex:0;], 1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:1;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:2;]}], select=[user_id_0,name,proctime,order_id,user_id_1,rowtime,payment_id,price,user_id_2], rowType=[RecordType(VARCHAR(2147483647) user_id_0, VARCHAR(2147483647) name, TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) proctime, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id_1, TIMESTAMP(3) rowtime, VARCHAR(2147483647) payment_id, INTEGER price, VARCHAR(2147483647) user_id_2)])" }, { - "id" : 57, + "id" : 73, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -417,7 +428,7 @@ "outputType" : "ROW<`name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647) NOT NULL, `rowtime` TIMESTAMP(3), `price` INT>", "description" : "Calc(select=[name, order_id, rowtime, price])" }, { - "id" : 58, + "id" : 74, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -460,81 +471,81 @@ "description" : "Sink(table=[default_catalog.default_database.sink], fields=[name, order_id, rowtime, price])" } ], "edges" : [ { - "source" : 47, - "target" : 48, + "source" : 63, + "target" : 64, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 48, - "target" : 49, + "source" : 64, + "target" : 65, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 50, - "target" : 51, + "source" : 66, + "target" : 67, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 51, - "target" : 52, + "source" : 67, + "target" : 68, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 52, - "target" : 53, + "source" : 68, + "target" : 69, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 54, - "target" : 55, + "source" : 70, + "target" : 71, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 49, - "target" : 56, + "source" : 65, + "target" : 72, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 53, - "target" : 56, + "source" : 69, + "target" : 72, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 55, - "target" : 56, + "source" : 71, + "target" : 72, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 56, - "target" : 57, + "source" : 72, + "target" : 73, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 57, - "target" : 58, + "source" : 73, + "target" : 74, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-join-with-time-attributes-with-restore/savepoint/_metadata index f11caa837fe444830bc7f7d829291d09406cecf7..c83557089f50deafd5f31b92579c0da6fefc6bd7 100644 GIT binary patch delta 1534 zcmc&!?Q0xG9Nvu>bJUYs5|WhA#HO12GTzS4?96OTluKyowK;2(>q{EWb9Q$&UXr^c zcbBuNVrm7kAhdCOL1>W_Dt_=&R;pHN{o>0v@r7cdffT<5g9=d)lAg0|0{#W<51wJ? zH_zQ(*cul4hddm>t0FCTM<2+n8^5S(?@{MNNor)Y2sN^$@u{LK-fjG&XDN2_R%ayn)6IqV{?Rl|S2(R}Z8n8e_}KJ3 zlTqdi#0?wU9v0BEZQCdE;q((TZ~D10(HXDsV%#4e^F*-}KSkoqpDtFOCncj&LCk;- z#3w6mezIt&7@mly<0DgrTp3Wv3}QeNF^Gvl2*o&F;uSG5o-33ALfmyB@N7yv%DAac zjyn!=soDdCB^bc09tyyGVgcmX$R(B~kcZ@QdILmLQ)A`a0rjULYG$-3HM99J14dLT zrx#21lQst&tr)j=Xp{Evws~lZ!@0 z4)>e2k17{vZSwv8riT_TkL!-sqn%)T!Bag$X{L7lkrZpFly2g+mx2 zp-sV42qR>ucx7utwR$HJj+-84p@-RR5AR-@U;C{4`gf%3$6cq_j>?JlouP$q?XGR1 zNYN;|O}nu8QS;sv`fv5AFE4ETeYS7Q>Ptr=^2)%DBvIT5L7R48X&p^|e7t&1^mMfL zy@elBCvVwZaOe{U+YYvv?>io`uxV20W7ozAoA<=-CRcT4*c3P}qySs>+yf4`2!Kck z3^6j9Zw1TgCOt_=sFQ5!{3PPY{p;(8Uw-Gn!4G@qF8@!bLk*jlHp0wzeb-VoZWGM0 z!x=^#a_e4{wQ|1$4PWWE5t z93nKMH4vJ8VQU5O&85{FnJ;d}HU`sOon4W+#kcQxqp^jCt&-wwDOVUZU;6FZr>}o= z?i{W?lX_!0p*Z!kSN7fQ#zqlxpku4&H5?M&09QQ#K^s6!A%T!nf~?@Yo_{2mKCxEw EA4t38lK=n! delta 1484 zcmc(f?{5@E7{_;4>lF|U6hg~c)0UzHGw#mL&n;B z(?$X98xnlMF(#rfR5Y4Kj6|+6h8Oz6#5cUsDD{P+CdQvZqmh^xrOrZQ_zPf?eP*6# zXXe?@lkYxXd~BaTZ6DcWx9fw08@3ZTZfztm-w-ubDT&Gy5Y=j7qO2;NfA4<%uAe&6 zXGm_{p{I6zq?dXBwwoR-l&1!Is>bFBAGcD3=M!h*&B51Pfz+4zv!8zT!toFF>Y>LD z=(tc+1MaM-xbbu`QkANUVz*K(MYt@>cn-l()`EbR%1@SR70--`8F$Q`td|Ql05L`s zaY7(Rh;zcs>{i9Bnwc(?Yd~V8Xb3z8l}BS3MuBt@gHpzX04`t*a5oDDkmexp;>c47 zQcAI~tlpeuSvTn3p<7z%o!Qx{E>owGgA-#Xp_ z+q&B2AOvAA!U`zCU>aZ^n{qIv07ej!fFmY^w5&U3Tcy<9ZMM@odE?8Dp1O67?dadv z2>tcmefk8QZCzY{zBA=Ct~XV4^zm(r`n$~Tt&Y9_acjTTRq`3+kX-?NGp=upI znd%$fGu%Bl`AYNLLNnQDo`bg4BN3M(_L%Quh6QFYVuFew4q&7N4`B23j;k(X-+TmD zTmml=1mKN|`9Q=JfshLi;n?RP4fP@WUp=}boh6<@CEf~Z=8cZC7uxUbe138M>7kv4 zd#*smL=cD~8WWG>5b^*^Y%~xdg<=|D|5~8N5ecOEy@AgJGpN*$fuM?jz>|>-J)>XY zC8*8(`Ygc-DPAG{?5(MRmzLgJck%lRZ{Pmg`-@kQMpy_ckqAV@B!w_m%212YXDXIl z;%h+~gp~M7L?8@cLdqiu0_Ib|y}&dkWXhoYgtTv^8~I5$@-KCxUp{%_$F`9NerUWi S=ls#PdcyRUEv;zZpU!^-BF!%V diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json index 5105a4410d939..aca670d1ff1d9 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/plan/three-way-left-outer-join-with-restore.json @@ -1,7 +1,7 @@ { "flinkVersion" : "2.1", "nodes" : [ { - "id" : 10, + "id" : 13, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -22,7 +22,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Users]], fields=[user_id, name])" }, { - "id" : 11, + "id" : 14, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -35,7 +35,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id]])" }, { - "id" : 12, + "id" : 15, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -56,7 +56,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[user_id, order_id])" }, { - "id" : 13, + "id" : 16, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -69,7 +69,80 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id]])" }, { - "id" : 14, + "id" : 17, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null + }, + "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<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(user_id = user_id0)], select=[user_id, name, user_id0, order_id], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" + }, { + "id" : 18, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Calc(select=[user_id, name, order_id])" + }, { + "id" : 19, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[user_id]])" + }, { + "id" : 20, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -90,7 +163,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, Payments]], fields=[user_id, payment_id])" }, { - "id" : 15, + "id" : 21, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -103,96 +176,41 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[user_id]])" }, { - "id" : 16, - "type" : "stream-exec-multi-join_1", - "joinTypes" : [ "INNER", "LEFT", "LEFT" ], - "joinConditions" : [ { - "kind" : "LITERAL", - "value" : true, - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "BOOLEAN" - } ], - "joinAttributeMap" : { - "1" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 0, - "rightInputId" : 1, - "rightFieldIndex" : 0 - } ], - "2" : [ { - "leftInputId" : 0, - "leftFieldIndex" : 0, - "rightInputId" : 2, - "rightFieldIndex" : 0 - } ] + "id" : 22, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "LEFT", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : null }, - "inputUniqueKeys" : [ [ ], [ ], [ ] ], "state" : [ { "index" : 0, "ttl" : "0 ms", - "name" : "input-state-0" + "name" : "leftState" }, { "index" : 1, "ttl" : "0 ms", - "name" : "input-state-1" - }, { - "index" : 2, - "ttl" : "0 ms", - "name" : "input-state-2" + "name" : "rightState" } ], "inputProperties" : [ { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", "priority" : 0 }, { "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 1 - }, { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] + "type" : "UNKNOWN" }, "damBehavior" : "PIPELINED", - "priority" : 2 + "priority" : 0 } ], - "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `user_id1` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", - "description" : "MultiJoin(joinFilter=[true], joinTypes=[[INNER, LEFT, LEFT]], joinConditions=[[true, =($0, $2), =($0, $4)]], joinAttributeMap=[{1=[LeftInputId:0;LeftFieldIndex:0;RightInputId:1;RightFieldIndex:0;], 2=[LeftInputId:0;LeftFieldIndex:0;RightInputId:2;RightFieldIndex:0;]}], select=[user_id,name,user_id0,order_id,user_id1,payment_id], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) name, VARCHAR(2147483647) user_id0, VARCHAR(2147483647) order_id, VARCHAR(2147483647) user_id1, VARCHAR(2147483647) payment_id)])" + "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `user_id0` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", + "description" : "Join(joinType=[LeftOuterJoin], where=[(user_id = user_id0)], select=[user_id, name, order_id, user_id0, payment_id], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])" }, { - "id" : 17, + "id" : 23, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -204,11 +222,11 @@ "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 3, + "inputIndex" : 2, "type" : "VARCHAR(2147483647)" }, { "kind" : "INPUT_REF", - "inputIndex" : 5, + "inputIndex" : 4, "type" : "VARCHAR(2147483647)" } ], "condition" : null, @@ -222,7 +240,7 @@ "outputType" : "ROW<`user_id` VARCHAR(2147483647), `name` VARCHAR(2147483647), `order_id` VARCHAR(2147483647), `payment_id` VARCHAR(2147483647)>", "description" : "Calc(select=[user_id, name, order_id, payment_id])" }, { - "id" : 18, + "id" : 24, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -265,60 +283,81 @@ "description" : "Sink(table=[default_catalog.default_database.sink], fields=[user_id, name, order_id, payment_id])" } ], "edges" : [ { - "source" : 10, - "target" : 11, + "source" : 13, + "target" : 14, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 12, - "target" : 13, + "source" : 15, + "target" : 16, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { "source" : 14, - "target" : 15, + "target" : 17, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 11, - "target" : 16, + "source" : 16, + "target" : 17, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 13, - "target" : 16, + "source" : 17, + "target" : 18, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 15, - "target" : 16, + "source" : 18, + "target" : 19, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 16, - "target" : 17, + "source" : 20, + "target" : 21, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 17, - "target" : 18, + "source" : 19, + "target" : 22, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 22, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" } ] -} +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-multi-join_1/three-way-left-outer-join-with-restore/savepoint/_metadata index f0288325eeeb97c37f52f63977613c1e4dbbea34..7b6e48d89ced35e4f05bd26650727d4125cb1515 100644 GIT binary patch delta 2031 zcmd5-ZD<@t7~b8hZAw1w(vP$;Nj=gcHOb!1%G_wi@D|Y;n1x<+&~T zYfmrjJo@7K#}D1TNz*i4&BMADwFP1>K{kMl67DB$?7EaG5X7}O6!KVS$JkR&-1SE@ z`FHJCFk zj2uEt8@U#C0j0p@afcJ`+72{r;=85|xNlO#xaktxX9QE>8ICJ==u0xux#m6u6fp$K z#4esb{zb#*ufFrsS9=#WJ}|d$88N|_ONr+}?s4XDSI{KDP(bKnzzKG4J~8UsOn5*j zPM&Almd{P<184%p7C?#|+hDLr>^(DbuN4Spm3e(|}NYi3req%C(+J`cIF*eM+ zei8H2slg|dk%|qlJd5QjMUVI{XuC-nsYps>xwpuyawykQp zgFMz2lWk^g?Ao;;tTamUAry#=it^cb_x<&szB2a1=Gm_+=RdstFZsf2OY()!+#308 zLa|GzV+t1BDp(~cg@TzJb7J{|@?<5(nigqSOl~aSB|TL5XxA?x`Q2TW1-`vm|6eJ2 zTcAWDkWP=vs~yp`%3QA7vfchR(2)3DEkx8T4p@R`1BlA!_e34_a((naN0g#94a$RU zb#mIQYp!Tlh29O_pq|}g)j#vcOTR5$IXOKs_sx!N7f#Mj0nmu-AYtsDtSW-b-=DZ} zrg30Ak@{qn{pH88DX>oVg7&eB(#IQ*l<}=Hep|w(+A{7e<7gRY%DA9w(odyDBA#cN zxIdH*+G6})H?l|nODBUI5gTIPv^Y{1zUK?o$pkiS)np;2UeboPrfphZ7=%X~LtC{f iIk~4!hC3?c38ON!HS~8IXe)yKLb#(vFZAxcZ2SqEnm}d% delta 1983 zcmdT^U1(fI6rOv#*^O=zW3s7TOqy()YO1~6nLBgu%xy5qnmi=ICfKS-V_5EQ-8TE< z?xrSvNE$&fXsS3bg-TWQK}11WUy1}@M4`5C6)FX5Km>)_6vQYJ*SVWri=^#a9vqlC zGiS~@-^`ga-_rZiDL=UeE8Mf zZD%73#Vf6&002q!QxfQ6j_bM})G^jDvs8>U!$O21N*snTG5K;}Y#w{@{8ZW@87o6h zVq{xI!_d>2*>o`vMLU{{KNUYVokAj`3AL>0#q1JQ}0SRVBhi(p1Ep_(J>7Eeg)O5?%lsbXqWM4CYQBxpo> zGVN=WV?y=bXP3U|pLk_4&FafN=RdSdH-n$Za`$*C7V4AaJySduf;_vkZEG)M8py=N zvlT)tLm?2miivGiTnk#d%TyBwu>rtCi=~glyXCfRpyk#LlvE}+T~uJ{TH|v8-q~}Z zb@L|T*Bbxe?~$vExj|jToHt)3vND{a+1g0S-uh(>3#vs0a=>!MO_huo=F@U98MgXFPF!6OGOAKv7lq-nxoe`y8j)so(-9C814}PCGP<_@fS#hF@CL;Oz?YH3h+}>0v+qo>2 zPDS1f+$JUJ|1Tx&cS{Lh(Od7oYZ@*F8tTMbi$CoD;v+n}KN=nI6@F_LeMhzT2+7T8 zgrAId1c5A8R>gQ&%qp#B+$C238ZYhN$InIe|6;ryjqV+&_=utA=HvKowT3n(3;geB ztcK@1-Vtw;V=S(5H_$u`efok``nte+`(B%Qsr&gGYn|WyIlCsjYV>8Bm|+XMOb`oG zRiZQ3^C29#m9@kofjGQ%py<70pA~aSRQ+GQ>W8?W!L%r|bzt zCqW3&q%9WvvS-S_{QO1J+Lr?ljviedpz?h%IPQj zAD&hc-j%zAZLxTQwl_< Date: Fri, 27 Mar 2026 13:59:14 +0100 Subject: [PATCH 6/6] [FLINK-38576][table] Fix spotless --- .../planner/plan/rules/logical/JoinToMultiJoinRule.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java index 922ff718ec78e..3dc3b76740b65 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/JoinToMultiJoinRule.java @@ -20,8 +20,6 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; -import org.apache.flink.table.planner.hint.FlinkHints; -import org.apache.flink.table.planner.hint.StateTtlHint; import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; import org.apache.flink.table.planner.plan.utils.IntervalJoinUtil; @@ -40,10 +38,6 @@ import org.apache.calcite.rel.core.Join; import org.apache.calcite.rel.core.JoinInfo; import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.TableFunctionScan; -import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.core.Values; -import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalSnapshot; import org.apache.calcite.rel.rules.CoreRules; @@ -67,12 +61,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.table.planner.hint.StateTtlHint.STATE_TTL; import static org.apache.flink.table.planner.plan.utils.MultiJoinUtil.createJoinAttributeMap; /**