rdblue commented on a change in pull request #2022:
URL: https://github.com/apache/iceberg/pull/2022#discussion_r561433683



##########
File path: 
spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -303,25 +305,107 @@ public void 
testSingleUnconditionalDeleteDisbleCountCheck() throws NoSuchTableEx
   }
 
   @Test
-  public void testSingleConditionalDeleteCountCheck() throws 
NoSuchTableException {
-    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, 
"emp-id-6"));
-    append(sourceName, new Employee(1, "emp-id-1"), new Employee(1, 
"emp-id-1"),
-           new Employee(2, "emp-id-2"), new Employee(6, "emp-id-6"));
+  public void testIdentityPartition()  {
+    writeModes.forEach(mode -> {
+      removeTables();
+      sql("CREATE TABLE %s (id INT, dep STRING) USING iceberg PARTITIONED BY 
(identity(dep))", targetName);
+      initTable(targetName);
+      setWriteMode(targetName, mode);
+      createAndInitSourceTable(sourceName);
+      append(targetName, new Employee(1, "emp-id-one"), new Employee(6, 
"emp-id-6"));
+      append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, 
"emp-id-1"), new Employee(6, "emp-id-6"));
+
+      String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+              "USING " + sourceName + " AS source \n" +
+              "ON target.id = source.id \n" +
+              "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * \n" +
+              "WHEN MATCHED AND target.id = 6 THEN DELETE \n" +
+              "WHEN NOT MATCHED AND source.id = 2 THEN INSERT * ";
+
+      sql(sqlText, "");
+      assertEquals("Should have expected rows",
+              ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2")),
+              sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+    });
+  }
 
-    String sqlText = "MERGE INTO %s AS target " +
-           "USING %s AS source " +
-           "ON target.id = source.id " +
-           "WHEN MATCHED AND target.id = 1 THEN DELETE " +
-           "WHEN NOT MATCHED AND source.id = 2 THEN INSERT * ";
+  @Test
+  public void testDaysTransform() {
+    writeModes.forEach(mode -> {
+      removeTables();
+      sql("CREATE TABLE %s (id INT, ts timestamp) USING iceberg PARTITIONED BY 
(days(ts))", targetName);
+      initTable(targetName);
+      setWriteMode(targetName, mode);
+      sql("CREATE TABLE %s (id INT, ts timestamp) USING iceberg", sourceName);
+      initTable(sourceName);
+      sql("INSERT INTO " + targetName + " VALUES (1, timestamp('2001-01-01 
00:00:00'))," +
+              "(6, timestamp('2001-01-06 00:00:00'))");
+      sql("INSERT INto " + sourceName + " VALUES (2, timestamp('2001-01-02 
00:00:00'))," +
+              "(1, timestamp('2001-01-01 00:00:00'))," +
+              "(6, timestamp('2001-01-06 00:00:00'))");
+
+      String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+              "USING " + sourceName + " AS source \n" +
+              "ON target.id = source.id \n" +
+              "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * \n" +
+              "WHEN MATCHED AND target.id = 6 THEN DELETE \n" +
+              "WHEN NOT MATCHED AND source.id = 2 THEN INSERT * ";
+
+      sql(sqlText, "");
+      assertEquals("Should have expected rows",
+              ImmutableList.of(row(1, "2001-01-01 00:00:00"), row(2, 
"2001-01-02 00:00:00")),
+              sql("SELECT id, CAST(ts AS STRING) FROM %s ORDER BY id ASC NULLS 
LAST", targetName));
+    });
+  }
 
-    String tabName = catalogName + "." + "default.target";
-    String errorMsg = "The same row of target table `" + tabName + "` was 
identified more than\n" +
-            " once for an update, delete or insert operation of the MERGE 
statement.";
-    AssertHelpers.assertThrows("Should complain ambiguous row in target",
-           SparkException.class, errorMsg, () -> sql(sqlText, targetName, 
sourceName));
-    assertEquals("Target should be unchanged",
-           ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")),
-           sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  @Test
+  public void testBucketExpression() {
+    writeModes.forEach(mode -> {
+      removeTables();
+      sql("CREATE TABLE %s (id INT, dep STRING) USING iceberg" +
+              " CLUSTERED BY (dep) INTO 2 BUCKETS", targetName);
+      initTable(targetName);
+      setWriteMode(targetName, mode);
+      createAndInitSourceTable(sourceName);
+      append(targetName, new Employee(1, "emp-id-one"), new Employee(6, 
"emp-id-6"));
+      append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, 
"emp-id-1"), new Employee(6, "emp-id-6"));
+      String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+              "USING " + sourceName + " AS source \n" +
+              "ON target.id = source.id \n" +
+              "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * \n" +
+              "WHEN MATCHED AND target.id = 6 THEN DELETE \n" +
+              "WHEN NOT MATCHED AND source.id = 2 THEN INSERT * ";
+
+      sql(sqlText, "");
+      assertEquals("Should have expected rows",
+              ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2")),
+              sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+    });
+  }
+
+  @Test
+  public void testPartitionedAndOrderedTable() {

Review comment:
       Where does this set the table ordering? I would expect it to run `ALTER 
TABLE %s WRITE ORDERED BY ...`




----------------------------------------------------------------
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.

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