-
Notifications
You must be signed in to change notification settings - Fork 360
chore: Improve clarity when using CatalogConfigs in CatalogBuilder
#1873
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
base: main
Are you sure you want to change the base?
Changes from 5 commits
3071456
104ae32
400c6b6
dc2dde4
15e3d2d
46091ff
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -56,18 +56,20 @@ const CARGO_PKG_VERSION: &str = env!("CARGO_PKG_VERSION"); | |
| const PATH_V1: &str = "v1"; | ||
|
|
||
| /// Builder for [`RestCatalog`]. | ||
| #[derive(Debug)] | ||
| pub struct RestCatalogBuilder(RestCatalogConfig); | ||
|
|
||
| impl Default for RestCatalogBuilder { | ||
| fn default() -> Self { | ||
| Self(RestCatalogConfig { | ||
| name: None, | ||
| uri: "".to_string(), | ||
| warehouse: None, | ||
| props: HashMap::new(), | ||
| client: None, | ||
| }) | ||
| #[derive(Debug, Default)] | ||
| pub struct RestCatalogBuilder { | ||
| name: Option<String>, | ||
| uri: Option<String>, | ||
| warehouse: Option<String>, | ||
| props: HashMap<String, String>, | ||
| client: Option<Client>, | ||
| } | ||
|
|
||
| impl RestCatalogBuilder { | ||
| /// Configures the catalog with a custom HTTP client. | ||
| pub fn with_client(mut self, client: Client) -> Self { | ||
| self.client = Some(client); | ||
| self | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -79,56 +81,55 @@ impl CatalogBuilder for RestCatalogBuilder { | |
| name: impl Into<String>, | ||
| props: HashMap<String, String>, | ||
| ) -> impl Future<Output = Result<Self::C>> + Send { | ||
| self.0.name = Some(name.into()); | ||
| self.name = Some(name.into()); | ||
|
|
||
| if props.contains_key(REST_CATALOG_PROP_URI) { | ||
| self.0.uri = props | ||
| .get(REST_CATALOG_PROP_URI) | ||
| .cloned() | ||
| .unwrap_or_default(); | ||
| self.uri = props.get(REST_CATALOG_PROP_URI).cloned(); | ||
| } | ||
|
|
||
| if props.contains_key(REST_CATALOG_PROP_WAREHOUSE) { | ||
| self.0.warehouse = props.get(REST_CATALOG_PROP_WAREHOUSE).cloned() | ||
| self.warehouse = props.get(REST_CATALOG_PROP_WAREHOUSE).cloned(); | ||
| } | ||
|
|
||
| // Collect other remaining properties | ||
| self.0.props = props | ||
| self.props = props | ||
| .into_iter() | ||
| .filter(|(k, _)| k != REST_CATALOG_PROP_URI && k != REST_CATALOG_PROP_WAREHOUSE) | ||
| .collect(); | ||
|
|
||
| let result = { | ||
| if self.0.name.is_none() { | ||
| Err(Error::new( | ||
| ErrorKind::DataInvalid, | ||
| "Catalog name is required", | ||
| )) | ||
| } else if self.0.uri.is_empty() { | ||
| Err(Error::new( | ||
| async move { | ||
| let name = self | ||
| .name | ||
| .ok_or_else(|| Error::new(ErrorKind::DataInvalid, "Catalog name is required"))?; | ||
|
|
||
| let uri = self | ||
| .uri | ||
| .ok_or_else(|| Error::new(ErrorKind::DataInvalid, "Catalog uri is required"))?; | ||
|
|
||
| if uri.is_empty() { | ||
| return Err(Error::new( | ||
| ErrorKind::DataInvalid, | ||
| "Catalog uri is required", | ||
| )) | ||
| } else { | ||
| Ok(RestCatalog::new(self.0)) | ||
| "Catalog uri cannot be empty", | ||
| )); | ||
| } | ||
| }; | ||
|
|
||
| std::future::ready(result) | ||
| } | ||
| } | ||
| let config = RestCatalogConfig { | ||
| name: Some(name), | ||
| uri, | ||
| warehouse: self.warehouse, | ||
| props: self.props, | ||
| client: self.client, | ||
| }; | ||
|
|
||
| impl RestCatalogBuilder { | ||
| /// Configures the catalog with a custom HTTP client. | ||
| pub fn with_client(mut self, client: Client) -> Self { | ||
| self.0.client = Some(client); | ||
| self | ||
| Ok(RestCatalog::new(config)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /// Rest catalog configuration. | ||
| #[derive(Clone, Debug, TypedBuilder)] | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we still need this
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just some questions for this one. Would this be breaking? Should I change the builder functionality for RestCatalog to be similar to the other catalogs (ex. using the load function.? Do you know why the builder for this one is different from the other catalogs?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it will not breaking since it's not public api. Is there any place actually using the generated builder of RestCatalogConfig? |
||
| pub(crate) struct RestCatalogConfig { | ||
| #[allow(dead_code)] // Stored for debugging and potential future use | ||
| #[builder(default, setter(strip_option))] | ||
| name: Option<String>, | ||
jonathanc-n marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.