dawidwys commented on code in PR #24253:
URL: https://github.com/apache/flink/pull/24253#discussion_r1482693914


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinTestPrograms.java:
##########
@@ -84,6 +130,36 @@ public class TemporalJoinTestPrograms {
                                     + "ON o.currency = r.currency ")
                     .build();
 
+    static final TableTestProgram TEMPORAL_JOIN_TABLE_JOIN_NESTED_KEY =
+            TableTestProgram.of(
+                            "temporal-join-table-join-nested-key",
+                            "validates temporal join with a table when the 
join keys comes from a nested row")
+                    .setupTableSource(ORDERS_WITH_NESTED_ID)
+                    .setupTableSource(RATES)
+                    .setupTableSink(AMOUNTS)
+                    .runSql(
+                            "INSERT INTO MySink "
+                                    + "SELECT amount * r.rate "
+                                    + "FROM OrdersNestedId AS o "
+                                    + "JOIN RatesHistory FOR SYSTEM_TIME AS OF 
o.rowtime AS r "
+                                    + "ON o.nested_row.currency = r.currency ")
+                    .build();
+
+    static final TableTestProgram TEMPORAL_JOIN_TABLE_JOIN_KEY_FROM_MAP =
+            TableTestProgram.of(
+                            "temporal-join-table-join-key-from-map",
+                            "validates temporal join with a table when the 
join key comes from a map value")
+                    .setupTableSource(ORDERS_WITH_NESTED_ID)
+                    .setupTableSource(RATES)
+                    .setupTableSink(AMOUNTS)
+                    .runSql(
+                            "INSERT INTO MySink "
+                                    + "SELECT amount * r.rate "
+                                    + "FROM OrdersNestedId AS o "
+                                    + "JOIN RatesHistory FOR SYSTEM_TIME AS OF 
o.rowtime AS r "
+                                    + "ON o.nested_map['currency'] = 
r.currency ")

Review Comment:
   I tried adding a case with a projection on the ride side of the input.
   
   The change introduced here works fine, unfortunately the query fails with a 
different condition unmet. The temporal table join requires the right side of 
the join (the versioned table) to be a `PRIMARY KEY`. `PRIMARY KEY` can only be 
defined on a source table, projections remove that constraint. That means the 
change improves the situation only for the left side of the join.
   
   I'll still add a case with a function on the left side. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to