@@ -26,10 +26,9 @@ use iceberg::io::{
2626use iceberg:: spec:: { TableMetadata , TableMetadataBuilder } ;
2727use iceberg:: table:: Table ;
2828use iceberg:: {
29- Catalog , Error , ErrorKind , MetadataLocation , Namespace , NamespaceIdent , Result , TableCommit ,
30- TableCreation , TableIdent ,
29+ Catalog , CatalogBuilder , Error , ErrorKind , MetadataLocation , Namespace , NamespaceIdent , Result ,
30+ TableCommit , TableCreation , TableIdent ,
3131} ;
32- use typed_builder:: TypedBuilder ;
3332
3433use crate :: error:: { from_aws_build_error, from_aws_sdk_error} ;
3534use crate :: utils:: {
@@ -40,15 +39,90 @@ use crate::{
4039 AWS_ACCESS_KEY_ID , AWS_REGION_NAME , AWS_SECRET_ACCESS_KEY , AWS_SESSION_TOKEN , with_catalog_id,
4140} ;
4241
43- #[ derive( Debug , TypedBuilder ) ]
42+ /// Glue catalog URI
43+ pub const GLUE_CATALOG_PROP_URI : & str = "uri" ;
44+ /// Glue catalog id
45+ pub const GLUE_CATALOG_PROP_CATALOG_ID : & str = "catalog_id" ;
46+ /// Glue catalog warehouse location
47+ pub const GLUE_CATALOG_PROP_WAREHOUSE : & str = "warehouse" ;
48+
49+ /// Builder for [`GlueCatalog`].
50+ #[ derive( Debug ) ]
51+ pub struct GlueCatalogBuilder ( GlueCatalogConfig ) ;
52+
53+ impl Default for GlueCatalogBuilder {
54+ fn default ( ) -> Self {
55+ Self ( GlueCatalogConfig {
56+ name : None ,
57+ uri : None ,
58+ catalog_id : None ,
59+ warehouse : "" . to_string ( ) ,
60+ props : HashMap :: new ( ) ,
61+ } )
62+ }
63+ }
64+
65+ impl CatalogBuilder for GlueCatalogBuilder {
66+ type C = GlueCatalog ;
67+
68+ fn load (
69+ mut self ,
70+ name : impl Into < String > ,
71+ props : HashMap < String , String > ,
72+ ) -> impl Future < Output = Result < Self :: C > > + Send {
73+ self . 0 . name = Some ( name. into ( ) ) ;
74+
75+ if props. contains_key ( GLUE_CATALOG_PROP_URI ) {
76+ self . 0 . uri = props. get ( GLUE_CATALOG_PROP_URI ) . cloned ( )
77+ }
78+
79+ if props. contains_key ( GLUE_CATALOG_PROP_CATALOG_ID ) {
80+ self . 0 . catalog_id = props. get ( GLUE_CATALOG_PROP_CATALOG_ID ) . cloned ( )
81+ }
82+
83+ if props. contains_key ( GLUE_CATALOG_PROP_WAREHOUSE ) {
84+ self . 0 . warehouse = props
85+ . get ( GLUE_CATALOG_PROP_WAREHOUSE )
86+ . cloned ( )
87+ . unwrap_or_default ( ) ;
88+ }
89+
90+ // Collect other remaining properties
91+ self . 0 . props = props
92+ . into_iter ( )
93+ . filter ( |( k, _) | {
94+ k != GLUE_CATALOG_PROP_URI
95+ && k != GLUE_CATALOG_PROP_CATALOG_ID
96+ && k != GLUE_CATALOG_PROP_WAREHOUSE
97+ } )
98+ . collect ( ) ;
99+
100+ async move {
101+ if self . 0 . name . is_none ( ) {
102+ return Err ( Error :: new (
103+ ErrorKind :: DataInvalid ,
104+ "Catalog name is required" ,
105+ ) ) ;
106+ }
107+ if self . 0 . warehouse . is_empty ( ) {
108+ return Err ( Error :: new (
109+ ErrorKind :: DataInvalid ,
110+ "Catalog warehouse is required" ,
111+ ) ) ;
112+ }
113+
114+ GlueCatalog :: new ( self . 0 ) . await
115+ }
116+ }
117+ }
118+
119+ #[ derive( Debug ) ]
44120/// Glue Catalog configuration
45- pub struct GlueCatalogConfig {
46- # [ builder ( default , setter ( strip_option ( fallback = uri_opt ) ) ) ]
121+ pub ( crate ) struct GlueCatalogConfig {
122+ name : Option < String > ,
47123 uri : Option < String > ,
48- #[ builder( default , setter( strip_option( fallback = catalog_id_opt) ) ) ]
49124 catalog_id : Option < String > ,
50125 warehouse : String ,
51- #[ builder( default ) ]
52126 props : HashMap < String , String > ,
53127}
54128
@@ -71,7 +145,7 @@ impl Debug for GlueCatalog {
71145
72146impl GlueCatalog {
73147 /// Create a new glue catalog
74- pub async fn new ( config : GlueCatalogConfig ) -> Result < Self > {
148+ async fn new ( config : GlueCatalogConfig ) -> Result < Self > {
75149 let sdk_config = create_sdk_config ( & config. props , config. uri . as_ref ( ) ) . await ;
76150 let mut file_io_props = config. props . clone ( ) ;
77151 if !file_io_props. contains_key ( S3_ACCESS_KEY_ID ) {
0 commit comments