liurenjie1024 commented on code in PR #1666:
URL: https://github.com/apache/iceberg-rust/pull/1666#discussion_r2367085406


##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -54,6 +53,87 @@ static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 
10 connections if n
 static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per 
connection to 10s before it is closed
 static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each 
connection to enabled prior to returning
 
+/// Builder for [`SqlCatalog`]
+#[derive(Debug)]
+pub struct SqlCatalogBuilder(SqlCatalogConfig);
+
+impl Default for SqlCatalogBuilder {
+    fn default() -> Self {
+        Self(SqlCatalogConfig {
+            uri: "".to_string(),
+            name: "".to_string(),
+            warehouse_location: "".to_string(),
+            file_io: FileIOBuilder::new_fs_io().build().unwrap(),
+            sql_bind_style: SqlBindStyle::DollarNumeric,
+            props: HashMap::new(),
+        })
+    }
+}
+
+impl SqlCatalogBuilder {
+    /// Configure the database URI
+    pub fn uri(mut self, uri: impl Into<String>) -> Self {
+        self.0.uri = uri.into();
+        self
+    }
+
+    /// Configure the warehouse location
+    pub fn warehouse_location(mut self, location: impl Into<String>) -> Self {
+        self.0.warehouse_location = location.into();
+        self
+    }
+
+    /// Configure the FileIO
+    pub fn file_io(mut self, file_io: FileIO) -> Self {

Review Comment:
   I don't think we need this. Current convention is to allow user to load 
`FileIO` for path and config.



##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -62,14 +142,13 @@ static TEST_BEFORE_ACQUIRE: bool = true; // Default the 
health-check of each con
 /// The options available for this parameter include:
 /// - `SqlBindStyle::DollarNumeric`: Binds SQL statements using `$1`, `$2`, 
etc., as placeholders. This is for PostgreSQL databases.
 /// - `SqlBindStyle::QuestionMark`: Binds SQL statements using `?` as a 
placeholder. This is for MySQL and SQLite databases.
-#[derive(Debug, TypedBuilder)]
+#[derive(Debug)]

Review Comment:
   Make this struct crate private? We should not expose it any more.



##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -54,6 +53,87 @@ static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 
10 connections if n
 static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per 
connection to 10s before it is closed
 static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each 
connection to enabled prior to returning
 
+/// Builder for [`SqlCatalog`]
+#[derive(Debug)]
+pub struct SqlCatalogBuilder(SqlCatalogConfig);
+
+impl Default for SqlCatalogBuilder {
+    fn default() -> Self {
+        Self(SqlCatalogConfig {
+            uri: "".to_string(),
+            name: "".to_string(),
+            warehouse_location: "".to_string(),
+            file_io: FileIOBuilder::new_fs_io().build().unwrap(),
+            sql_bind_style: SqlBindStyle::DollarNumeric,
+            props: HashMap::new(),
+        })
+    }
+}
+
+impl SqlCatalogBuilder {
+    /// Configure the database URI
+    pub fn uri(mut self, uri: impl Into<String>) -> Self {
+        self.0.uri = uri.into();
+        self
+    }
+
+    /// Configure the warehouse location
+    pub fn warehouse_location(mut self, location: impl Into<String>) -> Self {
+        self.0.warehouse_location = location.into();
+        self
+    }
+
+    /// Configure the FileIO
+    pub fn file_io(mut self, file_io: FileIO) -> Self {
+        self.0.file_io = file_io;
+        self
+    }
+
+    /// Configure the bound SQL Statement
+    pub fn sql_bind_style(mut self, sql_bind_style: SqlBindStyle) -> Self {
+        self.0.sql_bind_style = sql_bind_style;
+        self
+    }
+
+    /// Configure the any properties
+    pub fn props(mut self, props: HashMap<String, String>) -> Self {
+        self.0.props = props;
+        self
+    }
+
+    /// Set a new property on the property to be configured.
+    /// When multiple methods are executed with the same key,
+    /// the later-set value takes precedence.
+    pub fn prop(mut self, key: impl Into<String>, value: impl Into<String>) -> 
Self {
+        self.0.props.insert(key.into(), value.into());
+        self
+    }
+}
+
+impl CatalogBuilder for SqlCatalogBuilder {
+    type C = SqlCatalog;
+
+    fn load(
+        mut self,
+        name: impl Into<String>,
+        props: HashMap<String, String>,
+    ) -> impl Future<Output = Result<Self::C>> + Send {
+        let name = name.into();
+        self.0.props = props;

Review Comment:
   Another question is, since we provided two ways to config one entry, like 
uri, location. We need to add doc to explain the behavior and tests to verify 
it. Please take `RestCatalog` as reference.



##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -54,6 +53,87 @@ static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 
10 connections if n
 static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per 
connection to 10s before it is closed
 static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each 
connection to enabled prior to returning
 
+/// Builder for [`SqlCatalog`]
+#[derive(Debug)]
+pub struct SqlCatalogBuilder(SqlCatalogConfig);
+
+impl Default for SqlCatalogBuilder {
+    fn default() -> Self {
+        Self(SqlCatalogConfig {
+            uri: "".to_string(),
+            name: "".to_string(),
+            warehouse_location: "".to_string(),
+            file_io: FileIOBuilder::new_fs_io().build().unwrap(),
+            sql_bind_style: SqlBindStyle::DollarNumeric,
+            props: HashMap::new(),
+        })
+    }
+}
+
+impl SqlCatalogBuilder {
+    /// Configure the database URI
+    pub fn uri(mut self, uri: impl Into<String>) -> Self {
+        self.0.uri = uri.into();
+        self
+    }
+
+    /// Configure the warehouse location
+    pub fn warehouse_location(mut self, location: impl Into<String>) -> Self {
+        self.0.warehouse_location = location.into();
+        self
+    }
+
+    /// Configure the FileIO
+    pub fn file_io(mut self, file_io: FileIO) -> Self {
+        self.0.file_io = file_io;
+        self
+    }
+
+    /// Configure the bound SQL Statement
+    pub fn sql_bind_style(mut self, sql_bind_style: SqlBindStyle) -> Self {
+        self.0.sql_bind_style = sql_bind_style;
+        self
+    }
+
+    /// Configure the any properties
+    pub fn props(mut self, props: HashMap<String, String>) -> Self {
+        self.0.props = props;
+        self
+    }
+
+    /// Set a new property on the property to be configured.
+    /// When multiple methods are executed with the same key,
+    /// the later-set value takes precedence.
+    pub fn prop(mut self, key: impl Into<String>, value: impl Into<String>) -> 
Self {
+        self.0.props.insert(key.into(), value.into());
+        self
+    }
+}
+
+impl CatalogBuilder for SqlCatalogBuilder {
+    type C = SqlCatalog;
+
+    fn load(
+        mut self,
+        name: impl Into<String>,
+        props: HashMap<String, String>,
+    ) -> impl Future<Output = Result<Self::C>> + Send {
+        let name = name.into();
+        self.0.props = props;

Review Comment:
   Please add doc in `lib.rs` for the new approach for catalog loading.



##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -54,6 +53,87 @@ static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 
10 connections if n
 static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per 
connection to 10s before it is closed
 static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each 
connection to enabled prior to returning
 
+/// Builder for [`SqlCatalog`]
+#[derive(Debug)]
+pub struct SqlCatalogBuilder(SqlCatalogConfig);
+
+impl Default for SqlCatalogBuilder {
+    fn default() -> Self {
+        Self(SqlCatalogConfig {
+            uri: "".to_string(),
+            name: "".to_string(),
+            warehouse_location: "".to_string(),
+            file_io: FileIOBuilder::new_fs_io().build().unwrap(),
+            sql_bind_style: SqlBindStyle::DollarNumeric,
+            props: HashMap::new(),
+        })
+    }
+}
+
+impl SqlCatalogBuilder {
+    /// Configure the database URI
+    pub fn uri(mut self, uri: impl Into<String>) -> Self {
+        self.0.uri = uri.into();
+        self
+    }
+
+    /// Configure the warehouse location
+    pub fn warehouse_location(mut self, location: impl Into<String>) -> Self {
+        self.0.warehouse_location = location.into();
+        self
+    }
+
+    /// Configure the FileIO
+    pub fn file_io(mut self, file_io: FileIO) -> Self {
+        self.0.file_io = file_io;
+        self
+    }
+
+    /// Configure the bound SQL Statement
+    pub fn sql_bind_style(mut self, sql_bind_style: SqlBindStyle) -> Self {
+        self.0.sql_bind_style = sql_bind_style;
+        self
+    }
+
+    /// Configure the any properties
+    pub fn props(mut self, props: HashMap<String, String>) -> Self {
+        self.0.props = props;
+        self
+    }
+
+    /// Set a new property on the property to be configured.
+    /// When multiple methods are executed with the same key,
+    /// the later-set value takes precedence.
+    pub fn prop(mut self, key: impl Into<String>, value: impl Into<String>) -> 
Self {
+        self.0.props.insert(key.into(), value.into());
+        self
+    }
+}
+
+impl CatalogBuilder for SqlCatalogBuilder {
+    type C = SqlCatalog;
+
+    fn load(
+        mut self,
+        name: impl Into<String>,
+        props: HashMap<String, String>,
+    ) -> impl Future<Output = Result<Self::C>> + Send {
+        let name = name.into();
+        self.0.props = props;

Review Comment:
   We expect user to pass all necessary configs through this config, like uri, 
location, binding style. Please take other catalogs as a reference.



##########
crates/catalog/sql/src/catalog.rs:
##########
@@ -818,15 +897,13 @@ mod tests {
         let sql_lite_uri = format!("sqlite:{}", temp_path());
         sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap();
 
-        let config = SqlCatalogConfig::builder()
+        let builder = SqlCatalogBuilder::default()
             .uri(sql_lite_uri.to_string())
-            .name("iceberg".to_string())
             .warehouse_location(warehouse_location)
             .file_io(FileIOBuilder::new_fs_io().build().unwrap())
-            .sql_bind_style(SqlBindStyle::QMark)
-            .build();
+            .sql_bind_style(SqlBindStyle::QMark);

Review Comment:
   Our primary entrypoint of catalog builder should be using the `load` method, 
please add tests for it.



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

Reply via email to