rymurr commented on a change in pull request #1783:
URL: https://github.com/apache/iceberg/pull/1783#discussion_r540830882



##########
File path: 
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -56,48 +77,77 @@ public boolean supportsExternalMetadata() {
   }
 
   @Override
-  public SparkTable getTable(StructType schema, Transform[] partitioning, 
Map<String, String> options) {
-    // Get Iceberg table from options
-    Configuration conf = SparkSession.active().sessionState().newHadoopConf();
-    Table icebergTable = getTableAndResolveHadoopConfiguration(options, conf);
-
-    // Build Spark table based on Iceberg table, and return it
-    // Eagerly refresh the table before reading to ensure views containing 
this table show up-to-date data
-    return new SparkTable(icebergTable, schema, true);
+  public Table getTable(StructType schema, Transform[] partitioning, 
Map<String, String> options) {
+    Spark3Util.CatalogAndIdentifier catalogIdentifier = 
catalogAndIdentifier(new CaseInsensitiveStringMap(options));
+    CatalogPlugin catalog = catalogIdentifier.catalog();
+    Identifier ident = catalogIdentifier.identifier();
+
+    try {
+      if (catalog instanceof TableCatalog) {
+        return ((TableCatalog) catalog).loadTable(ident);
+      }
+    } catch (NoSuchTableException e) {
+      // throwing an iceberg NoSuchTableException because the Spark one is 
typed and cant be thrown from this interface
+      throw new org.apache.iceberg.exceptions.NoSuchTableException(e, "Cannot 
find table for %s.", ident);
+    }
+
+    // throwing an iceberg NoSuchTableException because the Spark one is typed 
and cant be thrown from this interface
+    throw new org.apache.iceberg.exceptions.NoSuchTableException("Cannot find 
table for %s.", ident);
   }
 
-  protected Table findTable(Map<String, String> options, Configuration conf) {
+  private Spark3Util.CatalogAndIdentifier 
catalogAndIdentifier(CaseInsensitiveStringMap options) {
     Preconditions.checkArgument(options.containsKey("path"), "Cannot open 
table: path is not set");
+    setupDefaultSparkCatalog();
     String path = options.get("path");
+    SparkSession spark = SparkSession.active();
+    CatalogManager catalogManager = spark.sessionState().catalogManager();
 
     if (path.contains("/")) {
-      HadoopTables tables = new HadoopTables(conf);
-      return tables.load(path);
-    } else {
-      HiveCatalog hiveCatalog = HiveCatalogs.loadCatalog(conf);
-      TableIdentifier tableIdentifier = TableIdentifier.parse(path);
-      return hiveCatalog.loadTable(tableIdentifier);
+      // contains a path. Return iceberg default catalog and a PathIdentifier
+      return new 
Spark3Util.CatalogAndIdentifier(catalogManager.catalog(DEFAULT_CATALOG_NAME),
+          new PathIdentifier(path));
     }
-  }
 
-  private Table getTableAndResolveHadoopConfiguration(Map<String, String> 
options, Configuration conf) {
-    // Overwrite configurations from the Spark Context with configurations 
from the options.
-    mergeIcebergHadoopConfs(conf, options);
-
-    Table table = findTable(options, conf);
+    final Spark3Util.CatalogAndIdentifier catalogAndIdentifier;
+    try {
+      catalogAndIdentifier = Spark3Util.catalogAndIdentifier(spark, path);
+    } catch (ParseException e) {
+      throw new IllegalArgumentException(String.format("Cannot parse path %s. 
It is not a valid SQL table", path), e);
+    }
 
-    // Set confs from table properties
-    mergeIcebergHadoopConfs(conf, table.properties());
+    if (catalogAndIdentifier.catalog().name().equals("spark_catalog") &&
+        !(catalogAndIdentifier.catalog() instanceof SparkSessionCatalog)) {
+      // catalog is a session catalog but does not support Iceberg. Use 
Iceberg instead.
+      return new 
Spark3Util.CatalogAndIdentifier(catalogManager.catalog(DEFAULT_CATALOG_NAME),
+          catalogAndIdentifier.identifier());
+    } else {
+      return catalogAndIdentifier;
+    }
+  }
 
-    // Re-overwrite values set in options and table properties but were not in 
the environment.
-    mergeIcebergHadoopConfs(conf, options);
+  @Override
+  public Identifier extractIdentifier(CaseInsensitiveStringMap options) {
+    return catalogAndIdentifier(options).identifier();
+  }
 
-    return table;
+  @Override
+  public String extractCatalog(CaseInsensitiveStringMap options) {
+    return catalogAndIdentifier(options).catalog().name();
   }
 
-  private static void mergeIcebergHadoopConfs(Configuration baseConf, 
Map<String, String> options) {
-    options.keySet().stream()
-        .filter(key -> key.startsWith("hadoop."))
-        .forEach(key -> baseConf.set(key.replaceFirst("hadoop.", ""), 
options.get(key)));
+  private static void setupDefaultSparkCatalog() {
+    SparkSession spark = SparkSession.active();

Review comment:
       I should have too. 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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to