You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/02/01 18:00:44 UTC

[GitHub] [iceberg] marton-bod opened a new pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

marton-bod opened a new pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191


   There are a couple of scenarios where the Hive DDL command `DROP TABLE` can produce an error unnecessarily.
   
   1. Currently on Hive4, DROP TABLE can result in a `FileNotFoundException` exception, because based on Hive2/Hive3 behaviour, the current assumption is that the `HiveIcebergMetaHook` has to clean up the metadata files after the Hive drop table has completed. However, in Hive4, if `external.table.purge=TRUE` is set, Hive will wipe the metadata folder as well.
   2. When creating a Hive table on top of an existing table in a different catalog, if we drop the table in Hive but the underlying table does not exist anymore (e.g. in the HadoopCatalog), we'd also end up with an exception.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570167338



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +

Review comment:
       Yes, good idea.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570095602



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +

Review comment:
       Would `testTables.createTable` be usable here?

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       Same here? `testTables.createTable`

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -64,6 +66,8 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS;
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA;

Review comment:
       If we import like this, can we change the other occurrences?

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       Looks good

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +570,46 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, SPEC,
+        FileFormat.PARQUET, ImmutableList.of());
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null,
+        HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {

Review comment:
       Just realized that probably this would be easier to read. Sorry for the late notice, should have caught earlier 😢 
   `Assume.assumeFalse("Not relevant for HiveCatalog", Catalogs.hiveCatalog(shell.getHiveConf()));`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#issuecomment-772350727


   @pvary Can you please take a look? Thanks!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570178678



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -64,6 +66,8 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS;
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA;

Review comment:
       If we import like this, can we change the other occurrences?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570167338



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +

Review comment:
       Yes, good idea.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       We need the `external.table.purge=TRUE` HMS table property to test the deletion scenario upon dropping the Hive table. However, this property only gets appended as an HMS table property in the MetaHook when using Hive Catalog, for non-HiveCatalog tables it doesn't get appended so I was using direct SQL. I've refactored `TestTables.createHiveTableSQL` to enable adding additional table properties to the Hive SQL.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -64,6 +66,8 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS;
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA;

Review comment:
       I changed it back so we don't use static imports.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +570,46 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, SPEC,
+        FileFormat.PARQUET, ImmutableList.of());
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null,
+        HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {

Review comment:
       Fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569297516



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       We already have a test case for `testCreateDropTable`, so I think this part is covered. Do I miss something?
   
   Could we create specific test cases instead for dropping the Iceberg table backing Hive table and then dropping the Hive table? (should only run for non-HiveCatalog tests)
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569313944



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       > We already have a test case for testCreateDropTable, so I think this part is covered. Do I miss something? Maybe we should create one where we add some data as well, if it is relevant
   
   That test case does not detect the difference of behaviour between Hive2/3 and Hive4 with regards to purge=TRUE, because if we create a table and not to do any additional snapshot-creating operations on the table, then `metadata.snapshots()` will give back an empty collection, hence the deleteFiles operation will not run during `dropTableData` (and you won't get FileNotFoundExc for Hive4). That's why I added the DROP TABLE to some tests where additional operations (like inserts) also take place.
   
   On the other hand, not sure why `metadata.snapshots()` gives back an empty collection in this case.
   
   > Could we create specific test cases instead for dropping the Iceberg table backing Hive table and then dropping the Hive table? (should only run for non-HiveCatalog tests)
   
   Yes, will do. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570359723



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +570,46 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, SPEC,
+        FileFormat.PARQUET, ImmutableList.of());
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null,
+        HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {

Review comment:
       Just realized that probably this would be easier to read. Sorry for the late notice, should have caught earlier 😢 
   `Assume.assumeFalse("Not relevant for HiveCatalog", Catalogs.hiveCatalog(shell.getHiveConf()));`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569303065



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -171,11 +171,22 @@ public void rollbackDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTabl
   @Override
   public void commitDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, boolean deleteData) {
     if (deleteData && deleteIcebergTable) {
-      if (!Catalogs.hiveCatalog(conf)) {
-        LOG.info("Dropping with purge all the data for table {}.{}", hmsTable.getDbName(), hmsTable.getTableName());
-        Catalogs.dropTable(conf, catalogProperties);
-      } else {
-        CatalogUtil.dropTableData(deleteIo, deleteMetadata);
+      try {
+        if (!Catalogs.hiveCatalog(conf)) {
+          LOG.info("Dropping with purge all the data for table {}.{}", hmsTable.getDbName(), hmsTable.getTableName());
+          Catalogs.dropTable(conf, catalogProperties);
+        } else {
+          // if metadata folder has been deleted already (Hive 4 behaviour for purge=TRUE), simply return
+          if (!deleteIo.newInputFile(deleteMetadata.location()).exists()) {

Review comment:
       Yes, that's cleaner :)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569333651



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       If it is not too inconvenient, create the snapshots directly on the iceberg tables, since it is faster than with hive inserts.
   
   I think it is better to have a specific test case for both cases then




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570522262



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +570,46 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    testTables.createTable(shell, identifier.name(), HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, SPEC,
+        FileFormat.PARQUET, ImmutableList.of());
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null,
+        HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {

Review comment:
       Fixed




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary merged pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary merged pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#issuecomment-773583796


   Thanks for your review @pvary!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569313944



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       > We already have a test case for testCreateDropTable, so I think this part is covered. Do I miss something? Maybe we should create one where we add some data as well, if it is relevant
   
   That test case does not detect the difference of behaviour between Hive2/3 and Hive4 with regards to purge=TRUE, because if we create a table and do not do any additional snapshot-creating operations on the table, then `metadata.snapshots()` will give back an empty collection, hence the deleteFiles operation will not run during `dropTableData` (and you won't get FileNotFoundExc for Hive4). That's why I added the DROP TABLE to some tests where additional operations (like inserts) also take place.
   
   On the other hand, not sure why `metadata.snapshots()` gives back an empty collection in this case.
   
   > Could we create specific test cases instead for dropping the Iceberg table backing Hive table and then dropping the Hive table? (should only run for non-HiveCatalog tests)
   
   Yes, will do. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570180397



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       Looks good




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569294619



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
##########
@@ -171,11 +171,22 @@ public void rollbackDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTabl
   @Override
   public void commitDropTable(org.apache.hadoop.hive.metastore.api.Table hmsTable, boolean deleteData) {
     if (deleteData && deleteIcebergTable) {
-      if (!Catalogs.hiveCatalog(conf)) {
-        LOG.info("Dropping with purge all the data for table {}.{}", hmsTable.getDbName(), hmsTable.getTableName());
-        Catalogs.dropTable(conf, catalogProperties);
-      } else {
-        CatalogUtil.dropTableData(deleteIo, deleteMetadata);
+      try {
+        if (!Catalogs.hiveCatalog(conf)) {
+          LOG.info("Dropping with purge all the data for table {}.{}", hmsTable.getDbName(), hmsTable.getTableName());
+          Catalogs.dropTable(conf, catalogProperties);
+        } else {
+          // if metadata folder has been deleted already (Hive 4 behaviour for purge=TRUE), simply return
+          if (!deleteIo.newInputFile(deleteMetadata.location()).exists()) {

Review comment:
       nit: Why not:
   ```
   if (deleteIo.newInputFile(deleteMetadata.location()).exists()) {
     CatalogUtil.dropTableData(deleteIo, deleteMetadata);
   }
   ```
   
   This way we can avoid returning in the middle of the code which is often hard to read, also maybe it is easier to understand.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570095602



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +

Review comment:
       Would `testTables.createTable` be usable here?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570169564



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       We need the `external.table.purge=TRUE` HMS table property to test the deletion scenario upon dropping the Hive table. However, this property only gets appended as an HMS table property in the MetaHook when using Hive Catalog, for non-HiveCatalog tables it doesn't get appended so I was using direct SQL. I've refactored `TestTables.createHiveTableSQL` to enable adding additional table properties to the Hive SQL.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570280641



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -64,6 +66,8 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS;
+import static org.apache.iceberg.mr.hive.HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA;

Review comment:
       I changed it back so we don't use static imports.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569297516



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       We already have a test case for `testCreateDropTable`, so I think this part is covered. Do I miss something? Maybe we should create one where we add some data as well, if it is relevant
   
   Could we create specific test cases instead for dropping the Iceberg table backing Hive table and then dropping the Hive table? (should only run for non-HiveCatalog tests)
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#issuecomment-773583796


   Thanks for your review @pvary!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#issuecomment-773942196


   Thanks for the fix @marton-bod 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r570097391



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
##########
@@ -568,6 +571,53 @@ public void testIcebergAndHmsTableProperties() throws TException, InterruptedExc
     }
   }
 
+  @Test
+  public void testDropTableWithAppendedData() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    shell.executeStatement(String.format("CREATE EXTERNAL TABLE %s STORED BY '%s' %s" +
+        "TBLPROPERTIES ('%s'='%s', '%s'='%s')",
+        identifier,
+        HiveIcebergStorageHandler.class.getName(),
+        testTables.locationForCreateTableSQL(identifier),
+        InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA),
+        InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)));
+
+    org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier);
+    testTables.appendIcebergTable(shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, CUSTOMER_RECORDS);
+
+    shell.executeStatement("DROP TABLE customers");
+  }
+
+  @Test
+  public void testDropHiveTableWithoutUnderlyingTable() throws IOException {
+    TableIdentifier identifier = TableIdentifier.of("default", "customers");
+
+    if (!Catalogs.hiveCatalog(shell.getHiveConf())) {
+      // Create the Iceberg table in non-HiveCatalog
+      testTables.createIcebergTable(shell.getHiveConf(), identifier.name(), CUSTOMER_SCHEMA, FileFormat.PARQUET,
+              CUSTOMER_RECORDS);
+
+      // Create Hive table on top
+      String tableLocation = testTables.locationForCreateTableSQL(identifier);

Review comment:
       Same here? `testTables.createTable`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569336951



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       Sure, I agree, will create two new tests for these specific scenarios




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2191: Hive: Avoid drop table related exceptions in MetaHook

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2191:
URL: https://github.com/apache/iceberg/pull/2191#discussion_r569313944



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java
##########
@@ -253,6 +253,7 @@ public void testCreateTableWithColumnSpecification() throws IOException {
         testTables.locationForCreateTableSQL(identifier);
     runCreateAndReadTest(identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
         PartitionSpec.unpartitioned(), data);
+    shell.executeStatement("DROP TABLE " + identifier);

Review comment:
       > We already have a test case for testCreateDropTable, so I think this part is covered. Do I miss something? Maybe we should create one where we add some data as well, if it is relevant
   
   That test case does not detect the difference of behaviour between Hive2/3 and Hive4 with regards to purge=TRUE, because if we create a table and do not do any additional snapshot-creating operations on the table, then `metadata.snapshots()` will give back an empty collection, hence the deleteFiles operation will not run during `dropTableData` (and you won't get FileNotFoundExc for Hive4). That's why I added the DROP TABLE to some tests where additional operations (like inserts) also take place.
   
   On the other hand, not sure if `metadata.snapshots()` giving back an empty collection in this case is the right behaviour.
   
   > Could we create specific test cases instead for dropping the Iceberg table backing Hive table and then dropping the Hive table? (should only run for non-HiveCatalog tests)
   
   Yes, will do. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org