chenjunjiedada commented on code in PR #4670:
URL: https://github.com/apache/iceberg/pull/4670#discussion_r863595348


##########
flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java:
##########
@@ -235,17 +236,59 @@ public void testPureInsertOnIdKey() throws Exception {
         )
     );
 
-    testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), 
elementsPerCheckpoint, expectedRecords);
+    testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), 
elementsPerCheckpoint, expectedRecords, false);
+  }
+
+  @Test
+  public void testUpsertOnIdKey() throws Exception {
+    List<List<Row>> elementsPerCheckpoint = ImmutableList.of(
+        ImmutableList.of(
+            insertRow(1, "aaa")
+        ),
+        ImmutableList.of(
+            updateBeforeRow(1, "aaa"),
+            updateAfterRow(1, "bbb")
+        ),
+        ImmutableList.of(
+            updateBeforeRow(1, "bbb"),
+            updateAfterRow(1, "ccc")
+        ),
+        ImmutableList.of(
+            updateBeforeRow(1, "ccc"),
+            updateAfterRow(1, "ddd"),
+            updateBeforeRow(1, "ddd"),
+            updateAfterRow(1, "eee")
+        )
+    );
+
+    List<List<Row>> expectedRecords = ImmutableList.of(
+        ImmutableList.of(insertRow(1, "aaa")),
+        ImmutableList.of(insertRow(1, "bbb")),
+        ImmutableList.of(insertRow(1, "ccc")),
+        ImmutableList.of(insertRow(1, "eee"))
+    );
+
+    if (!partitioned) {
+      testSqlChangeLog(TABLE_NAME, ImmutableList.of("id"), 
elementsPerCheckpoint, expectedRecords, true);
+    } else {
+      AssertHelpers.assertThrows("Should be error because equality field 
columns don't include all partition keys",
+          IllegalStateException.class, "should be included in equality fields",
+          () -> {
+            testSqlChangeLog(TABLE_NAME, ImmutableList.of("id"), 
elementsPerCheckpoint, expectedRecords, true);
+            return null;
+          });
+    }
   }
 
   private static Record record(int id, String data) {
     return SimpleDataUtil.createRecord(id, data);
   }
 
-  private Table createTable(String tableName, List<String> key, boolean 
isPartitioned) {
+  private Table createTable(String tableName, List<String> key, boolean 
isPartitioned, boolean upsertEnabled) {
     String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : "";
-    sql("CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) 
%s",
-        tableName, Joiner.on(',').join(key), partitionByCause);
+    sql("CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) 
%s " +
+            "WITH ('write-upsert-enabled'='%s')",
+        tableName, Joiner.on(',').join(key), partitionByCause, upsertEnabled ? 
"true" : "false");

Review Comment:
   The `write-upsert-enabled` is the option of the Flink logic table and the 
`write.upsert.enabled` is the property of the iceberg table, the separators are 
different. The unit test creates the table via the Flink catalog and thus 
respects the passed job level option `write-upsert-enabled`. 
   
   I just changed the option to align the source options in 
`FlinkConfigOptions`, I think that would be a better place to put source/sink 
options.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to