Skip to content
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

feat: expand ListingSchemaProvider to support register and deregister table #3150

Open
wants to merge 5 commits into
base: main
Choose a base branch
from

Conversation

Nordalf
Copy link
Contributor

@Nordalf Nordalf commented Jan 21, 2025

Description

The ListingSchemaProvider was missing the implementations of the register and deregister traits from the SchemaProvider. These are now implemented, and to meet requirements from remote object stores, the DeltaTableBuilder is now used to pass along the storage options.

The tables property has been modified to instead of having DashMap<String, String>, it is now using Mutex<DashMap<String, Arc<dyn TableProvider>>> to support downcast_ref and mutability elsewhere.

Inspiration from: Datafusion ListingSchemaProvider

Related Issue(s)

There is no related issues. I have a clone of this with extra modifications in my own project. I find it useful to have when adding a new schema and wanted to share this with others 👍

Documentation

None

@github-actions github-actions bot added the binding/rust Issues for the Rust crate label Jan 21, 2025
@Nordalf
Copy link
Contributor Author

Nordalf commented Jan 21, 2025

Open for discussion on this one. Have had this locally for a long time and wanted to share the branch. I use this quite a lot in my project and others may find it useful to their project.

@ion-elgreco
Copy link
Collaborator

@hntd187 can you take a look? :)

@@ -36,7 +36,7 @@ pub struct ListingSchemaProvider {
/// Underlying object store
store: Arc<dyn ObjectStore>,
/// A map of table names to a fully quilfied storage location
tables: DashMap<String, String>,
tables: Mutex<DashMap<String, Arc<dyn TableProvider>>>,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to wrap the DashMap in a Mutex? I thought that DashMap handles mutability / locking internally and can just be put in an Arc if it needs to be shared across threads?

Then again, I might be remembering that wrong.

Copy link
Collaborator

@hntd187 hntd187 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm supportive of this change, but I would like some changes to the internals.

@@ -54,7 +54,7 @@ impl ListingSchemaProvider {
Ok(Self {
authority: uri.to_string(),
store,
tables: DashMap::new(),
tables: Mutex::new(DashMap::new()),
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the necessity of the lock? Dashmap shouldn't need a locking mechanism, it's a direct replacement for things such as this.

self.tables.insert(table_name.to_string(), table_url);
let Ok(delta_table) = DeltaTableBuilder::from_uri(table_url)
.with_storage_options(self.storage_options.0.clone())
.load()
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we need to load every table on listing. You only need a list of tables, loading them gives you much more than that. Additionally, this might be a performance issue or cause a lot of unnecessary API calls if you have a very large catalog and never read 99% of the tables in it.

self.tables
.lock()
.expect("Can't lock tables")
.insert(name, table.clone());
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will replace the table provider if it currently exists, are you sure this is what you want?

.lock()
.expect("Can't lock tables")
.get(name)
.map(|t| t.clone())
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

would .cloned() suffice here instead?

self.tables.iter().map(|t| t.key().clone()).collect()
self.tables
.lock()
.expect("Can't lock tables")
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If there is a reason to keep the locking mechanism I think we should handle these errors instead of an expect panic when we can't acquire a lock.

@Nordalf
Copy link
Contributor Author

Nordalf commented Jan 21, 2025

Thanks for the great comments @roeap and @hntd187. I will take a look and handle them tomorrow 🚀

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
binding/rust Issues for the Rust crate
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants