rdblue commented on a change in pull request #1784:
URL: https://github.com/apache/iceberg/pull/1784#discussion_r527957914
##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseSparkAction.java
##########
@@ -128,33 +127,48 @@
return manifestDF.union(otherMetadataFileDF).union(manifestListDF);
}
+ // Attempt to use Spark3 Catalog resolution if available on the path
+ private static DynMethods.StaticMethod loadCatalogImpl = null;
+
+ private static Dataset<Row> loadCatalogMetadataTable(SparkSession spark,
String tableName, MetadataTableType type) {
+ if (loadCatalogImpl == null) {
+ try {
+ loadCatalogImpl = DynMethods.builder("loadCatalogMetadataTable")
+ .hiddenImpl("org.apache.iceberg.spark.Spark3Util",
+ SparkSession.class, String.class, MetadataTableType.class)
+ .buildStaticChecked();
+ } catch (NoSuchMethodException e) {
+ throw new IllegalArgumentException("Cannot find Spark3Util class but
Spark 3 is being used.", e);
+ }
+ }
+ return loadCatalogImpl.invoke(spark, tableName, type);
+ }
+
protected static Dataset<Row> loadMetadataTable(SparkSession spark, String
tableName, String tableLocation,
MetadataTableType type) {
- DataFrameReader noCatalogReader = spark.read().format("iceberg");
+ DataFrameReader dataFrameReader = spark.read().format("iceberg");
if (tableName.contains("/")) {
// Hadoop Table or Metadata location passed, load without a catalog
- return noCatalogReader.load(tableName + "#" + type);
+ return dataFrameReader.load(tableName + "#" + type);
}
- // Try catalog based name based resolution
- try {
- return spark.table(tableName + "." + type);
- } catch (Exception e) {
- if (!(e instanceof ParseException || e instanceof AnalysisException)) {
- // Rethrow unexpected exceptions
- throw e;
- }
- // Catalog based resolution failed, our catalog may be a
non-DatasourceV2 Catalog
- if (tableName.startsWith("hadoop.")) {
- // Try loading by location as Hadoop table without Catalog
- return noCatalogReader.load(tableLocation + "#" + type);
- } else if (tableName.startsWith("hive")) {
- // Try loading by name as a Hive table without Catalog
- return noCatalogReader.load(tableName.replaceFirst("hive\\.", "") +
"." + type);
- } else {
- throw new IllegalArgumentException(String.format(
- "Cannot find the metadata table for %s of type %s", tableName,
type));
+ // Try DSV2 catalog based name based resolution
+ if (spark.version().startsWith("3")) {
+ Dataset<Row> catalogMetadataTable = loadCatalogMetadataTable(spark,
tableName, type);
+ if (catalogMetadataTable != null) {
+ return catalogMetadataTable;
}
}
+ // Catalog based resolution failed, our catalog may be a non-DatasourceV2
Catalog
Review comment:
Nit: should have an empty line after the last block.
----------------------------------------------------------------
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]