rdblue commented on a change in pull request #1784:
URL: https://github.com/apache/iceberg/pull/1784#discussion_r526511929
##########
File path:
spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java
##########
@@ -110,4 +112,54 @@ public void testSparkCatalogHiveTable() throws
TableAlreadyExistsException, NoSu
results.contains("file:" + location + "/data/trashfile"));
}
+ @Test
+ public void testSparkSessionCatalogHadoopTable() throws Exception {
+ spark.conf().set("spark.sql.catalog.spark_catalog",
"org.apache.iceberg.spark.SparkSessionCatalog");
+ spark.conf().set("spark.sql.catalog.spark_catalog.type", "hadoop");
+ spark.conf().set("spark.sql.catalog.spark_catalog.warehouse",
tableLocation);
+ SparkSessionCatalog cat = (SparkSessionCatalog)
spark.sessionState().catalogManager().v2SessionCatalog();
+
+ String[] database = {"default"};
+ Identifier id = Identifier.of(database, "table");
+ Map<String, String> options = Maps.newHashMap();
+ Transform[] transforms = {};
+ cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+ SparkTable table = (SparkTable) cat.loadTable(id);
+
+ spark.sql("INSERT INTO default.table VALUES (1,1,1)");
+
+ String location = table.table().location().replaceFirst("file:", "");
+ new File(location + "/data/trashfile").createNewFile();
+
+ List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+ .olderThan(System.currentTimeMillis() + 1000).execute();
+ Assert.assertTrue("trash file should be removed",
+ results.contains("file:" + location + "/data/trashfile"));
+ }
+
+ @Test
+ public void testSparkSessionCatalogHiveTable() throws Exception {
+ spark.conf().set("spark.sql.catalog.spark_catalog",
"org.apache.iceberg.spark.SparkSessionCatalog");
+ spark.conf().set("spark.sql.catalog.spark_catalog.type", "hive");
+ SparkSessionCatalog cat = (SparkSessionCatalog)
spark.sessionState().catalogManager().v2SessionCatalog();
+
+ String[] database = {"default"};
+ Identifier id = Identifier.of(database, "sessioncattest");
+ Map<String, String> options = Maps.newHashMap();
+ Transform[] transforms = {};
+ cat.dropTable(id);
+ cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, options);
+ SparkTable table = (SparkTable) cat.loadTable(id);
+
+ spark.sql("INSERT INTO default.sessioncattest VALUES (1,1,1)");
+
+ String location = table.table().location().replaceFirst("file:", "");
+ new File(location + "/data/trashfile").createNewFile();
+
+ List<String> results = Actions.forTable(table.table()).removeOrphanFiles()
+ .olderThan(System.currentTimeMillis() + 1000).execute();
Review comment:
I think this test case only works because `HiveCatalogs` uses the value
of `hive.metastore.uris` from the environment's `hive-site.xml`. By removing
`spark_catalog` and then using the `DataFrameReader`, the Hive catalog from
`HiveCatalogs` is used, which has the same URI.
I think this is actually the right thing to do, but I would do it more
directly and obviously so that it is clear what is happening:
1. Get the session catalog from the catalog manager
2. If the session catalog is a SparkSessionCatalog, get the underlying
Iceberg catalog
3. Use the Iceberg catalog to load the metadata table, because it accepts
the full table identifier
----------------------------------------------------------------
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]