liurenjie1024 commented on code in PR #1873:
URL: https://github.com/apache/iceberg-rust/pull/1873#discussion_r2558624281
##########
crates/catalog/rest/src/catalog.rs:
##########
@@ -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)]
Review Comment:
I think it will not breaking since it's not public api. Is there any place
actually using the generated builder of RestCatalogConfig?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]