-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Custom catalogs from IcebergSource
#1783
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
Why not do something like the following, which can make the whole thing more backwards compatible: df.write.format("iceberg")
.mode("append")
.option("catalog-impl", "com.my.own.CatalogImpl")
.save("testing.foo") |
I was thinking similar but as per #1587 we don't want to hold catalog configs in the hadoop |
|
@jackye1995 I think I misunderstood your original comment. When you said: Did you mean then use I wonder if we could set a default catalog in I believe that this still causes an issue in Spark2 as it doesn't have the same (spark) catalog support. Thoughts? |
|
I agree that we will need to mimic the behavior of Here's what I think is the correct thing to do for table identifiers in Spark 3:
That way, we're just using the source to translate back to catalogs. The problem with this is the behavior when there is no catalog specified in the identifier. The current behavior is to use a Another option is to change the current behavior slightly and go with the "correct" logic for Spark 3 and delegate to the current catalog. That would mean we always have a catalog to delegate to without creating one. But the trade-off is that when the current catalog has changed, the table loaded by Last, how to handle path URIs: I talked with Anton and Russell about this yesterday and we think that we should make it so that every Spark catalog can load special path identifiers, just like we do today in What do you think? |
|
A couple of questions before we try to come up with a solution.
|
The purpose is to maintain compatibility with 2.4. If you have a working job that used IcebergSource to load a table from the Hive catalog, then that should continue to work.
The same way that we lookup catalogs in Spark: If the first part is a catalog, return it. Otherwise, use a default. The problem with this is how we want to maintain compatibility with Spark 2.4. In 2.4, the behavior is that the default catalog is the Hive session catalog and that can't be changed. In 3, I think there is a good argument that since the session catalog is the default unless you configure a different one (spark.sql.defaultCatalog) then we can respect the default catalog. |
+1 to that. I think we have enough info to get #1306 done. While it definitely makes sense to accept |
I like this approach but will it be backward compatible if the user does not set the session catalog as Iceberg session catalog? |
It's hard to define because we would be respecting a setting that wasn't present in 2.4. I think it's a reasonable thing to do as long as no catalog setting changes maintain compatibility with 2.4. Otherwise, the user is opting for different behavior by changing the default catalog. |
|
Thanks all for the input. So I think there is enough info that I can clean up this PR to use
Does that make sense? Am I missing anything? PS Any thoughts on custom catalogs in Spark2.4? I don't know what the migration plan looks like but I feel like Spark2.4 will be a thing for some time. |
I think that the problem earlier was that once you implement I think that will require that all of our catalogs support an identifier that contains a path and use that path to load the table.
We could do this in the |
Agreed. I was going to open a PR for that to merge before this one. Unless you want it all in 1 PR? |
|
Separate PRs works for me. I think you'll need to add that first, or else you won't be able to get tests passing. |
85ed069 to
2e5ab04
Compare
|
FYI this has been updated based on our discussions last week. Build is still failing but once #1843 is closed it should be good to go. |
spark3/src/test/java/org/apache/iceberg/spark/source/SetupSourceCatalog.java
Outdated
Show resolved
Hide resolved
spark3/src/test/java/org/apache/iceberg/spark/source/SetupSourceCatalog.java
Outdated
Show resolved
Hide resolved
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
Outdated
Show resolved
Hide resolved
2e5ab04 to
ac33ea3
Compare
| @@ -1,23 +0,0 @@ | |||
| /* | |||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A potential complication: SupportsCatalogOptions doesn't allow for specifying the schema. I don't know how much people rely on this feature but it is a breaking change for the IcebergSource
spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java
Outdated
Show resolved
Hide resolved
spark/src/test/java/org/apache/iceberg/spark/source/SetupSourceCatalog.java
Outdated
Show resolved
Hide resolved
| } 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); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the correct exception is the Spark exception since this is going to be called from Spark code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The Spark NoSuchTableException is typed which would change the interface. It appears there is no well defined way to return from this function w/o a table being found. It NPEs if you return null so an untyped exception seems to be the expected way to denote no table found. I thought the Iceberg NoSuchTableException was the best compromise here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Got it, I agree in that case. Thanks for explaining! You may want to add a comment here to explain in the code as well.
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
Outdated
Show resolved
Hide resolved
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
Outdated
Show resolved
Hide resolved
spark3/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
Outdated
Show resolved
Hide resolved
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
| if (ident.size() == 1) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: could you separate the control flow statements with a newline?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍
3aa4d29 to
a4c08d1
Compare
|
Thanks for checking, @xabriel! I'll merge this when tests are passing. |
|
Thanks for all your hard work on this, @rymurr! Great to have this working! |
The changes are mostly in spark3. They are necessitated by the catalog support introduced by apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
The changes are mostly in spark3. They are necessitated by the catalog support introduced in apache#1783. As the spark3 IcebergSource now implements SupportsCatalogOptions, DataFrameReader#load no longer calls IcebergSource#getTable but calls SparkCatalog#loadTable directly. In order for the SparkTable returned by SparkCatalog#loadTable(Identifier) to be aware of the snapshot, the information about the snapshot needs to be present in the Identifier. For this reason, we introduce a SnapshotAwareIdentifier interface extending Identifier. As SupportsCatalogOptions does not allow a schema to be specified (requested), SparkTable no longer needs a requestedSchema field, so some dead code is removed from it.
I wanted to start a conversation about how to make
IcebergSourcecompatible with custom Iceberg Catalogs in both Spark 2&3.This first attempt is meant to give a straw man for how this could be done. It uses code from
LookupCatalogto fetch a Catalog and Identifier from a path. We then try and extract an Iceberg table from it.This works by specifying the catalog(
iceberg_catalog) below in the path or by setting a default catalog in spark config.df.write.format("iceberg").mode("append").save("iceberg_catalog.testing.foo")Several problems:
LookupCatalogit would be nice to use that (or similar) directly but that would involve putting scala code in the code path.I am soliciting opinions on what might be a better way of doing custom catalogs, has anyone else been thinking about this?
cc @rdblue @jacques-n @laurentgo