Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 21 additions & 8 deletions crates/catalog/glue/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,18 @@ impl Default for GlueCatalogBuilder {
}
}

impl GlueCatalogBuilder {
/// Get a mutable reference to the catalog configuration.
pub(crate) fn catalog_config(&mut self) -> &mut GlueCatalogConfig {
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not a big fan of such a change. I think a better approach would be following:

pub struct GlueCatalogBuilder {
   name: Option<String>,
   ....
}

struct GlueCatalogConfig {
   name: String
}

This makes things easier to read, more importantly, the config class type safe. Builder struct need to store intermediate states, so it's reasonable to have it contains many optional fields. The config struct, which is finally built and verfied, should only contain valid states, e.g. it should no longer has unnecessary optional fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Made the changes in 400c6b6

&mut self.0
}

/// Consume the builder and return the catalog configuration.
pub(crate) fn into_config(self) -> GlueCatalogConfig {
self.0
}
}

impl CatalogBuilder for GlueCatalogBuilder {
type C = GlueCatalog;

Expand All @@ -73,25 +85,25 @@ impl CatalogBuilder for GlueCatalogBuilder {
name: impl Into<String>,
props: HashMap<String, String>,
) -> impl Future<Output = Result<Self::C>> + Send {
self.0.name = Some(name.into());
self.catalog_config().name = Some(name.into());

if props.contains_key(GLUE_CATALOG_PROP_URI) {
self.0.uri = props.get(GLUE_CATALOG_PROP_URI).cloned()
self.catalog_config().uri = props.get(GLUE_CATALOG_PROP_URI).cloned()
}

if props.contains_key(GLUE_CATALOG_PROP_CATALOG_ID) {
self.0.catalog_id = props.get(GLUE_CATALOG_PROP_CATALOG_ID).cloned()
self.catalog_config().catalog_id = props.get(GLUE_CATALOG_PROP_CATALOG_ID).cloned()
}

if props.contains_key(GLUE_CATALOG_PROP_WAREHOUSE) {
self.0.warehouse = props
self.catalog_config().warehouse = props
.get(GLUE_CATALOG_PROP_WAREHOUSE)
.cloned()
.unwrap_or_default();
}

// Collect other remaining properties
self.0.props = props
self.catalog_config().props = props
.into_iter()
.filter(|(k, _)| {
k != GLUE_CATALOG_PROP_URI
Expand All @@ -100,21 +112,22 @@ impl CatalogBuilder for GlueCatalogBuilder {
})
.collect();

let config = self.into_config();
async move {
if self.0.name.is_none() {
if config.name.is_none() {
return Err(Error::new(
ErrorKind::DataInvalid,
"Catalog name is required",
));
}
if self.0.warehouse.is_empty() {
if config.warehouse.is_empty() {
return Err(Error::new(
ErrorKind::DataInvalid,
"Catalog warehouse is required",
));
}

GlueCatalog::new(self.0).await
GlueCatalog::new(config).await
}
}
}
Expand Down
31 changes: 22 additions & 9 deletions crates/catalog/hms/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,18 @@ impl Default for HmsCatalogBuilder {
}
}

impl HmsCatalogBuilder {
/// Get a mutable reference to the catalog configuration.
pub(crate) fn catalog_config(&mut self) -> &mut HmsCatalogConfig {
&mut self.0
}

/// Consume the builder and return the catalog configuration.
pub(crate) fn into_config(self) -> HmsCatalogConfig {
self.0
}
}

impl CatalogBuilder for HmsCatalogBuilder {
type C = HmsCatalog;

Expand All @@ -74,28 +86,28 @@ impl CatalogBuilder for HmsCatalogBuilder {
name: impl Into<String>,
props: HashMap<String, String>,
) -> impl Future<Output = Result<Self::C>> + Send {
self.0.name = Some(name.into());
self.catalog_config().name = Some(name.into());

if props.contains_key(HMS_CATALOG_PROP_URI) {
self.0.address = props.get(HMS_CATALOG_PROP_URI).cloned().unwrap_or_default();
self.catalog_config().address = props.get(HMS_CATALOG_PROP_URI).cloned().unwrap_or_default();
}

if let Some(tt) = props.get(HMS_CATALOG_PROP_THRIFT_TRANSPORT) {
self.0.thrift_transport = match tt.to_lowercase().as_str() {
self.catalog_config().thrift_transport = match tt.to_lowercase().as_str() {
THRIFT_TRANSPORT_FRAMED => HmsThriftTransport::Framed,
THRIFT_TRANSPORT_BUFFERED => HmsThriftTransport::Buffered,
_ => HmsThriftTransport::default(),
};
}

if props.contains_key(HMS_CATALOG_PROP_WAREHOUSE) {
self.0.warehouse = props
self.catalog_config().warehouse = props
.get(HMS_CATALOG_PROP_WAREHOUSE)
.cloned()
.unwrap_or_default();
}

self.0.props = props
self.catalog_config().props = props
.into_iter()
.filter(|(k, _)| {
k != HMS_CATALOG_PROP_URI
Expand All @@ -104,24 +116,25 @@ impl CatalogBuilder for HmsCatalogBuilder {
})
.collect();

let config = self.into_config();
let result = {
if self.0.name.is_none() {
if config.name.is_none() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog name is required",
))
} else if self.0.address.is_empty() {
} else if config.address.is_empty() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog address is required",
))
} else if self.0.warehouse.is_empty() {
} else if config.warehouse.is_empty() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog warehouse is required",
))
} else {
HmsCatalog::new(self.0)
HmsCatalog::new(config)
}
};

Expand Down
41 changes: 26 additions & 15 deletions crates/catalog/rest/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,24 @@ impl Default for RestCatalogBuilder {
}
}

impl RestCatalogBuilder {
/// Get a mutable reference to the catalog configuration.
pub(crate) fn catalog_config(&mut self) -> &mut RestCatalogConfig {
&mut self.0
}

/// Consume the builder and return the catalog configuration.
pub(crate) fn into_config(self) -> RestCatalogConfig {
self.0
}

/// Configures the catalog with a custom HTTP client.
pub fn with_client(mut self, client: Client) -> Self {
self.catalog_config().client = Some(client);
self
}
}

impl CatalogBuilder for RestCatalogBuilder {
type C = RestCatalog;

Expand All @@ -79,53 +97,46 @@ 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.catalog_config().name = Some(name.into());

if props.contains_key(REST_CATALOG_PROP_URI) {
self.0.uri = props
self.catalog_config().uri = props
.get(REST_CATALOG_PROP_URI)
.cloned()
.unwrap_or_default();
}

if props.contains_key(REST_CATALOG_PROP_WAREHOUSE) {
self.0.warehouse = props.get(REST_CATALOG_PROP_WAREHOUSE).cloned()
self.catalog_config().warehouse = props.get(REST_CATALOG_PROP_WAREHOUSE).cloned()
}

// Collect other remaining properties
self.0.props = props
self.catalog_config().props = props
.into_iter()
.filter(|(k, _)| k != REST_CATALOG_PROP_URI && k != REST_CATALOG_PROP_WAREHOUSE)
.collect();

let config = self.into_config();
let result = {
if self.0.name.is_none() {
if config.name.is_none() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog name is required",
))
} else if self.0.uri.is_empty() {
} else if config.uri.is_empty() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog uri is required",
))
} else {
Ok(RestCatalog::new(self.0))
Ok(RestCatalog::new(config))
}
};

std::future::ready(result)
}
}

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

/// Rest catalog configuration.
#[derive(Clone, Debug, TypedBuilder)]
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we still need this TypedBuilder?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

The 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 {
Expand Down
31 changes: 21 additions & 10 deletions crates/catalog/s3tables/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ pub const S3TABLES_CATALOG_PROP_ENDPOINT_URL: &str = "endpoint_url";

/// S3Tables catalog configuration.
#[derive(Debug)]
struct S3TablesCatalogConfig {
pub(crate) struct S3TablesCatalogConfig {
/// Catalog name.
name: Option<String>,
/// Unlike other buckets, S3Tables bucket is not a physical bucket, but a virtual bucket
Expand Down Expand Up @@ -82,6 +82,16 @@ impl Default for S3TablesCatalogBuilder {

/// Builder methods for [`S3TablesCatalog`].
impl S3TablesCatalogBuilder {
/// Get a mutable reference to the catalog configuration.
pub(crate) fn catalog_config(&mut self) -> &mut S3TablesCatalogConfig {
&mut self.0
}

/// Consume the builder and return the catalog configuration.
pub(crate) fn into_config(self) -> S3TablesCatalogConfig {
self.0
}

/// Configure the catalog with a custom endpoint URL (useful for local testing/mocking).
///
/// # Behavior with Properties
Expand All @@ -91,13 +101,13 @@ impl S3TablesCatalogBuilder {
/// This follows the general pattern where properties specified in the `load()` method
/// have higher priority than builder method configurations.
pub fn with_endpoint_url(mut self, endpoint_url: impl Into<String>) -> Self {
self.0.endpoint_url = Some(endpoint_url.into());
self.catalog_config().endpoint_url = Some(endpoint_url.into());
self
}

/// Configure the catalog with a pre-built AWS SDK client.
pub fn with_client(mut self, client: aws_sdk_s3tables::Client) -> Self {
self.0.client = Some(client);
self.catalog_config().client = Some(client);
self
}

Expand All @@ -110,7 +120,7 @@ impl S3TablesCatalogBuilder {
/// This follows the general pattern where properties specified in the `load()` method
/// have higher priority than builder method configurations.
pub fn with_table_bucket_arn(mut self, table_bucket_arn: impl Into<String>) -> Self {
self.0.table_bucket_arn = table_bucket_arn.into();
self.catalog_config().table_bucket_arn = table_bucket_arn.into();
self
}
}
Expand All @@ -124,41 +134,42 @@ impl CatalogBuilder for S3TablesCatalogBuilder {
props: HashMap<String, String>,
) -> impl Future<Output = Result<Self::C>> + Send {
let catalog_name = name.into();
self.0.name = Some(catalog_name.clone());
self.catalog_config().name = Some(catalog_name.clone());

if props.contains_key(S3TABLES_CATALOG_PROP_TABLE_BUCKET_ARN) {
self.0.table_bucket_arn = props
self.catalog_config().table_bucket_arn = props
.get(S3TABLES_CATALOG_PROP_TABLE_BUCKET_ARN)
.cloned()
.unwrap_or_default();
}

if props.contains_key(S3TABLES_CATALOG_PROP_ENDPOINT_URL) {
self.0.endpoint_url = props.get(S3TABLES_CATALOG_PROP_ENDPOINT_URL).cloned();
self.catalog_config().endpoint_url = props.get(S3TABLES_CATALOG_PROP_ENDPOINT_URL).cloned();
}

// Collect other remaining properties
self.0.props = props
self.catalog_config().props = props
.into_iter()
.filter(|(k, _)| {
k != S3TABLES_CATALOG_PROP_TABLE_BUCKET_ARN
&& k != S3TABLES_CATALOG_PROP_ENDPOINT_URL
})
.collect();

let config = self.into_config();
async move {
if catalog_name.trim().is_empty() {
Err(Error::new(
ErrorKind::DataInvalid,
"Catalog name cannot be empty",
))
} else if self.0.table_bucket_arn.is_empty() {
} else if config.table_bucket_arn.is_empty() {
Err(Error::new(
ErrorKind::DataInvalid,
"Table bucket ARN is required",
))
} else {
S3TablesCatalog::new(self.0).await
S3TablesCatalog::new(config).await
}
}
}
Expand Down
Loading
Loading