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



##########
File path: 
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -56,48 +61,62 @@ 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) {
+    String catalogName = extractCatalog(new CaseInsensitiveStringMap(options));
+    Identifier ident = extractIdentifier(new 
CaseInsensitiveStringMap(options));
+    CatalogManager catalogManager = 
SparkSession.active().sessionState().catalogManager();
+    CatalogPlugin catalog = catalogManager.catalog(catalogName);
+    try {
+      if (catalog instanceof TableCatalog) {
+        return ((TableCatalog) catalog).loadTable(ident);
+      }
+    } catch (NoSuchTableException e) {
+      throw new org.apache.iceberg.exceptions.NoSuchTableException(e, "Cannot 
find table for %s.", ident);
+    }
+    throw new org.apache.iceberg.exceptions.NoSuchTableException("Cannot find 
table for %s.", ident);
   }
 
-  protected Table findTable(Map<String, String> options, Configuration conf) {
+  private Pair<String, TableIdentifier> 
tableIdentifier(CaseInsensitiveStringMap options) {
+    CatalogManager catalogManager = 
SparkSession.active().sessionState().catalogManager();
+    Namespace defaultNamespace = 
Namespace.of(catalogManager.currentNamespace());
     Preconditions.checkArgument(options.containsKey("path"), "Cannot open 
table: path is not set");
     String path = options.get("path");
-
-    if (path.contains("/")) {
-      HadoopTables tables = new HadoopTables(conf);
-      return tables.load(path);
+    List<String> ident;
+    try {
+      ident = 
scala.collection.JavaConverters.seqAsJavaList(SparkSession.active().sessionState().sqlParser().parseMultipartIdentifier(path));
+    } catch (ParseException e) {
+      try {
+        ident = 
scala.collection.JavaConverters.seqAsJavaList(SparkSession.active().sessionState().sqlParser().parseMultipartIdentifier(String.format("`%s`",
 path)));
+      } catch (ParseException ignored) {
+        throw new RuntimeException(e);
+      }
+    }
+    if (ident.size() == 1) {
+      return Pair.of(null, TableIdentifier.of(defaultNamespace, ident.get(0)));
+    } else if (ident.size() == 2) {
+      if (catalogManager.isCatalogRegistered(ident.get(0))) {
+        return Pair.of(ident.get(0), TableIdentifier.of(defaultNamespace, 
ident.get(1))); //todo what if path?
+      } else {
+        return Pair.of(null, TableIdentifier.of(ident.toArray(new String[0])));
+      }
     } else {
-      HiveCatalog hiveCatalog = HiveCatalogs.loadCatalog(conf);
-      TableIdentifier tableIdentifier = TableIdentifier.parse(path);
-      return hiveCatalog.loadTable(tableIdentifier);
+      if (catalogManager.isCatalogRegistered(ident.get(0))) {
+        return Pair.of(ident.get(0), TableIdentifier.of(ident.subList(1, 
ident.size()).toArray(new String[0])));
+      } else {
+        return Pair.of(null, TableIdentifier.of(ident.toArray(new String[0])));
+      }
     }
   }
 
-  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);
-
-    // Set confs from table properties
-    mergeIcebergHadoopConfs(conf, table.properties());
-
-    // Re-overwrite values set in options and table properties but were not in 
the environment.
-    mergeIcebergHadoopConfs(conf, options);
-
-    return table;
+  @Override
+  public Identifier extractIdentifier(CaseInsensitiveStringMap options) {
+    TableIdentifier tableIdentifier = tableIdentifier(options).second();
+    return Identifier.of(tableIdentifier.namespace().levels(), 
tableIdentifier.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)));
+  @Override
+  public String extractCatalog(CaseInsensitiveStringMap options) {
+    String catalogName = tableIdentifier(options).first();
+    return (catalogName == null) ? 
SupportsCatalogOptions.super.extractCatalog(options) : catalogName;

Review comment:
       The default implementation always returns `spark_catalog`, not the 
current catalog. Since we want to use the current catalog when it isn't defined 
in the identifier, `tableIdentifier(options)` should fill it in. That would 
simplify this logic because it should always be non-null.




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