aokolnychyi commented on a change in pull request #4141:
URL: https://github.com/apache/iceberg/pull/4141#discussion_r810426206



##########
File path: 
spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -224,4 +236,60 @@ public void 
testConcurrentExpireSnapshotsWithInvalidInput() {
             catalogName, tableIdent, -1));
 
   }
+
+  @Test
+  public void testExpireDeleteFiles() throws Exception {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg 
TBLPROPERTIES" +
+        "('format-version'='2', 'write.delete.mode'='merge-on-read')", 
tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 
tableName);
+    sql("DELETE FROM %s WHERE id=1", tableName);
+
+    Table table = Spark3Util.loadIcebergTable(spark, tableName);
+
+    Assert.assertEquals("Should have 1 delete manifest", 1, 
deleteManifests(table).size());
+    Assert.assertEquals("Should have 1 delete file", 1, 
deleteFiles(table).size());
+    Path deleteManifestPath = new 
Path(deleteManifests(table).iterator().next().path());
+    Path deleteFilePath = new 
Path(String.valueOf(deleteFiles(table).iterator().next().path()));
+
+    sql("CALL %s.system.rewrite_data_files(table => '%s', options => map" +

Review comment:
       nit: I think it would be easier to read if the args were on separate 
lines like in a few other places.

##########
File path: 
spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -224,4 +235,62 @@ public void 
testConcurrentExpireSnapshotsWithInvalidInput() {
             catalogName, tableIdent, -1));
 
   }
+
+  @Test
+  public void testExpireDeleteFiles() throws Exception {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg 
TBLPROPERTIES" +
+        "('format-version'='2', 'write.delete.mode'='merge-on-read')", 
tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 
tableName);
+    sql("DELETE FROM %s WHERE id=1", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    table.refresh();
+
+    Assert.assertEquals("Should have 1 delete manifest", 1, 
deleteManifests(table).size());
+    Assert.assertEquals("Should have 1 delete file", 1, 
deleteFiles(table).size());
+    Path deleteManifestPath = new 
Path(deleteManifests(table).iterator().next().path());
+    Path deleteFilePath = new 
Path(String.valueOf(deleteFiles(table).iterator().next().path()));
+
+    sql("CALL %s.system.rewrite_data_files(table => '%s', options => map" +
+            "('delete-file-threshold','1', 'use-starting-sequence-number', 
'false'))",
+        catalogName, tableIdent);
+
+    table.refresh();
+
+    sql("INSERT INTO TABLE %s VALUES (5, 'e')", tableName); // this txn moves 
the file to the DELETED state
+    sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn 
removes the file reference

Review comment:
       Makes sense

##########
File path: 
spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -224,4 +236,60 @@ public void 
testConcurrentExpireSnapshotsWithInvalidInput() {
             catalogName, tableIdent, -1));
 
   }
+
+  @Test
+  public void testExpireDeleteFiles() throws Exception {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg 
TBLPROPERTIES" +
+        "('format-version'='2', 'write.delete.mode'='merge-on-read')", 
tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 
tableName);

Review comment:
       I am afraid we don't know the number of files that this insert will 
produce. That's why ID = 1 may end up in a separate file (unlikely but 
possible). I think it would be safer to use a typed `Dataset` and 
`SimpleRecord`. That way, we can call `coalesce(1)` before writing to make sure 
we produce only 1 file and a subsequent metadata delete is not possible. If we 
write just a single file with ID = 1, the DELETE operation below will be a 
metadata operation and the test will fail.

##########
File path: 
spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -224,4 +236,60 @@ public void 
testConcurrentExpireSnapshotsWithInvalidInput() {
             catalogName, tableIdent, -1));
 
   }
+
+  @Test
+  public void testExpireDeleteFiles() throws Exception {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg 
TBLPROPERTIES" +
+        "('format-version'='2', 'write.delete.mode'='merge-on-read')", 
tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 
tableName);
+    sql("DELETE FROM %s WHERE id=1", tableName);
+
+    Table table = Spark3Util.loadIcebergTable(spark, tableName);
+
+    Assert.assertEquals("Should have 1 delete manifest", 1, 
deleteManifests(table).size());
+    Assert.assertEquals("Should have 1 delete file", 1, 
deleteFiles(table).size());
+    Path deleteManifestPath = new 
Path(deleteManifests(table).iterator().next().path());
+    Path deleteFilePath = new 
Path(String.valueOf(deleteFiles(table).iterator().next().path()));
+
+    sql("CALL %s.system.rewrite_data_files(table => '%s', options => map" +
+            "('delete-file-threshold','1', 'use-starting-sequence-number', 
'false'))",
+        catalogName, tableIdent);
+    table.refresh();
+
+    sql("INSERT INTO TABLE %s VALUES (5, 'e')", tableName); // this txn moves 
the file to the DELETED state
+    sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn 
removes the file reference
+
+    table.refresh();
+    Assert.assertEquals("Should have no delete manifests", 0, 
deleteManifests(table).size());
+    Assert.assertEquals("Should have no delete files", 0, 
deleteFiles(table).size());
+
+    FileSystem localFs = FileSystem.getLocal(new Configuration());
+    Assert.assertTrue("Delete manifest should still exist", 
localFs.exists(deleteManifestPath));
+    Assert.assertTrue("Delete file should still exist", 
localFs.exists(deleteFilePath));
+
+    Timestamp currentTimestamp = 
Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+    sql("CALL %s.system.expire_snapshots(" +
+            "older_than => TIMESTAMP '%s'," +
+            "table => '%s'," +
+            "retain_last => 1)",
+        catalogName, currentTimestamp, tableIdent);
+
+    Assert.assertFalse("Delete manifest should be removed", 
localFs.exists(deleteManifestPath));
+    Assert.assertFalse("Delete file should be removed", 
localFs.exists(deleteFilePath));
+  }
+
+  private Set<ManifestFile> deleteManifests(Table table) {

Review comment:
       nit: this may be simplified a bit?
   
   ```
     private List<ManifestFile> deleteManifests(Table table) {
       return table.currentSnapshot().deleteManifests();
     }
   
     private Set<DeleteFile> deleteFiles(Table table) {
       Set<DeleteFile> deleteFiles = Sets.newHashSet();
   
       for (FileScanTask task : table.newScan().planFiles()) {
         deleteFiles.addAll(task.deletes());
       }
   
       return deleteFiles;
     }
   ```




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