odysa commented on code in PR #229: URL: https://github.com/apache/iceberg-rust/pull/229#discussion_r1513340583
########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { Review Comment: I suggest using a config struct like HiveCatalog ```rust /// Hive metastore Catalog configuration. #[derive(Debug, TypedBuilder)] pub struct HmsCatalogConfig { address: String, thrift_transport: HmsThriftTransport, } ``` And RestCatalog ```rust /// Rest catalog configuration. #[derive(Debug, TypedBuilder)] pub struct RestCatalogConfig { uri: String, #[builder(default, setter(strip_option))] warehouse: Option<String>, #[builder(default)] props: HashMap<String, String>, } ``` ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { + install_default_drivers(); + + let mut connection = AnyConnectOptions::connect( + &AnyConnectOptions::from_url(&url.try_into()?).map_err(from_sqlx_error)?, + ) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_tables ( + catalog_name text not null, + table_namespace text not null, + table_name text not null, + metadata_location text not null, Review Comment: metadata_location is nullable as in [pyiceberg](https://github.com/apache/iceberg-python/blob/be8152994efc148021d6c6d5bca150e97c1676c0/pyiceberg/catalog/sql.py#L83C4-L83C90) and [Java Iceberg ](https://github.com/apache/iceberg/blob/2519ab43d654927802cc02e19c917ce90e8e0265/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L103-L104) ```python class IcebergTables(SqlCatalogBaseTable): __tablename__ = "iceberg_tables" .... metadata_location: Mapped[Optional[str]] = mapped_column(String(1000), nullable=True) previous_metadata_location: Mapped[Optional[str]] = mapped_column(String(1000), nullable=True) ``` ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { + install_default_drivers(); + + let mut connection = AnyConnectOptions::connect( + &AnyConnectOptions::from_url(&url.try_into()?).map_err(from_sqlx_error)?, + ) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_tables ( + catalog_name text not null, Review Comment: Can we use VARCHAR to be consistent with Pyiceberg and Java [Iceberg](https://github.com/apache/iceberg/blob/2519ab43d654927802cc02e19c917ce90e8e0265/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L94C3-L102C38)? ``` java "CREATE TABLE " + CATALOG_TABLE_VIEW_NAME + "(" + CATALOG_NAME + " VARCHAR(255) NOT NULL," + TABLE_NAMESPACE + " VARCHAR(255) NOT NULL," + TABLE_NAME + " VARCHAR(255) NOT NULL," ``` ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { + install_default_drivers(); + + let mut connection = AnyConnectOptions::connect( + &AnyConnectOptions::from_url(&url.try_into()?).map_err(from_sqlx_error)?, + ) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_tables ( + catalog_name text not null, + table_namespace text not null, + table_name text not null, + metadata_location text not null, + previous_metadata_location text, + primary key (catalog_name, table_namespace, table_name) + );", + ) + .execute(&mut **txn) + .await + }) + }) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_namespace_properties ( + catalog_name text not null, + namespace text not null, + property_key text, + property_value text, + primary key (catalog_name, namespace, property_key) + );", + ) + .execute(&mut **txn) + .await + }) + }) + .await + .map_err(from_sqlx_error)?; + + Ok(SqlCatalog { + name: name.to_owned(), + connection: Arc::new(Mutex::new(connection)), + storage, + cache: Arc::new(DashMap::new()), + }) + } +} + +#[derive(Debug)] +struct TableRef { + table_namespace: String, + table_name: String, + metadata_location: String, + _previous_metadata_location: Option<String>, +} + +fn query_map(row: &AnyRow) -> std::result::Result<TableRef, sqlx::Error> { + Ok(TableRef { + table_namespace: row.try_get(0)?, + table_name: row.try_get(1)?, + metadata_location: row.try_get(2)?, + _previous_metadata_location: row.try_get::<String, _>(3).map(Some).or_else(|err| { + if let sqlx::Error::ColumnDecode { + index: _, + source: _, + } = err + { + Ok(None) + } else { + Err(err) + } + })?, + }) +} + +#[async_trait] +impl Catalog for SqlCatalog { + async fn list_namespaces( + &self, + _parent: Option<&NamespaceIdent>, + ) -> Result<Vec<NamespaceIdent>> { + let mut connection = self.connection.lock().await; + let rows = connection.transaction(|txn|{ + let name = self.name.clone(); + Box::pin(async move { + sqlx::query(&format!("select distinct table_namespace from iceberg_tables where catalog_name = '{}';",&name)).fetch_all(&mut **txn).await + })}).await.map_err(from_sqlx_error)?; + let iter = rows.iter().map(|row| row.try_get::<String, _>(0)); + + Ok(iter + .map(|x| { + x.and_then(|y| { + NamespaceIdent::from_vec( + y.split('.').map(ToString::to_string).collect::<Vec<_>>(), + ) + .map_err(|err| sqlx::Error::Decode(Box::new(err))) + }) + }) + .collect::<std::result::Result<_, sqlx::Error>>() + .map_err(from_sqlx_error)?) + } + + async fn create_namespace( + &self, + _namespace: &NamespaceIdent, + _properties: HashMap<String, String>, + ) -> Result<Namespace> { + todo!() + } + + async fn get_namespace(&self, _namespace: &NamespaceIdent) -> Result<Namespace> { + todo!() + } + + async fn namespace_exists(&self, _namespace: &NamespaceIdent) -> Result<bool> { + todo!() + } + + async fn update_namespace( + &self, + _namespace: &NamespaceIdent, + _properties: HashMap<String, String>, + ) -> Result<()> { + todo!() + } + + async fn drop_namespace(&self, _namespace: &NamespaceIdent) -> Result<()> { + todo!() + } + + async fn list_tables(&self, namespace: &NamespaceIdent) -> Result<Vec<TableIdent>> { + let mut connection = self.connection.lock().await; + let rows = connection.transaction(|txn|{ + let name = self.name.clone(); + let namespace = namespace.encode_in_url(); + Box::pin(async move { + sqlx::query(&format!("select table_namespace, table_name, metadata_location, previous_metadata_location from iceberg_tables where catalog_name = '{}' and table_namespace = '{}';",&name, &namespace)).fetch_all(&mut **txn).await + })}).await.map_err(from_sqlx_error)?; + let iter = rows.iter().map(query_map); + + Ok(iter + .map(|x| { + x.and_then(|y| { + let namespace = NamespaceIdent::from_vec( + y.table_namespace + .split('.') + .map(ToString::to_string) + .collect::<Vec<_>>(), + ) + .map_err(|err| sqlx::Error::Decode(Box::new(err)))?; + Ok(TableIdent::new(namespace, y.table_name)) + }) + }) + .collect::<std::result::Result<_, sqlx::Error>>() + .map_err(from_sqlx_error)?) + } + + async fn stat_table(&self, identifier: &TableIdent) -> Result<bool> { + let mut connection = self.connection.lock().await; + let rows = connection.transaction(|txn|{ + let catalog_name = self.name.clone(); + let namespace = identifier.namespace().encode_in_url(); + let name = identifier.name().to_string(); + Box::pin(async move { + sqlx::query(&format!("select table_namespace, table_name, metadata_location, previous_metadata_location from iceberg_tables where catalog_name = '{}' and table_namespace = '{}' and table_name = '{}';",&catalog_name, + &namespace, + &name)).fetch_all(&mut **txn).await + })}).await.map_err(from_sqlx_error)?; + let mut iter = rows.iter().map(query_map); + + Ok(iter.next().is_some()) + } + + async fn drop_table(&self, _identifier: &TableIdent) -> Result<()> { + todo!() + } + + async fn load_table(&self, identifier: &TableIdent) -> Result<Table> { + let metadata_location = { Review Comment: Do we need to check the cache first? Given that it's inserted later. ########## crates/catalog/sql/Cargo.toml: ########## @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "iceberg-catalog-sql" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +rust-version = { workspace = true } + +categories = ["database"] +description = "Apache Iceberg Rust Sql Catalog" +repository = { workspace = true } +license = { workspace = true } +keywords = ["iceberg", "sql", "catalog"] + +[dependencies] +anyhow = "1" +async-trait = { workspace = true } +chrono = { workspace = true } +dashmap = "5.5.3" +futures = { workspace = true } +iceberg = { workspace = true } +log = "0.4.20" +opendal = { workspace = true } +serde = { workspace = true } +serde_derive = { workspace = true } +serde_json = { workspace = true } +sqlx = { version = "0.7.2", features = ["runtime-tokio-rustls", "any", "sqlite", "postgres", "mysql"], default-features = false } Review Comment: SQLX [doc](https://github.com/launchbadge/sqlx/blob/main/README.md) says this feature is SOFT-DEPRECATED. > you should use the separate runtime and TLS features as the combination features may be removed in the future. Also, is it possible to make this crate runtime agnostic? Sqlx is compatible with tokio, async-std,and actix runtime. Maybe we can give users runtime choices? ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, Review Comment: I suggest using the connection pool. A single connection with a mutex lock is probably a huge bottleneck. See [Pool in Sqlx](https://docs.rs/sqlx/latest/sqlx/struct.Pool.html) Do we need a connection pool here? cc @Fokko @liurenjie1024 @Xuanwo ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { + install_default_drivers(); + + let mut connection = AnyConnectOptions::connect( + &AnyConnectOptions::from_url(&url.try_into()?).map_err(from_sqlx_error)?, + ) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_tables ( Review Comment: Can we declare some static const variables? ```rust static CATALOG_TABLE_NAME: &str = "iceberg_tables" ... ``` like those from Java [JdbcCatalog](https://github.com/apache/iceberg/blob/2519ab43d654927802cc02e19c917ce90e8e0265/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L49-L55) ```java // Catalog Table & View static final String CATALOG_TABLE_VIEW_NAME = "iceberg_tables"; static final String CATALOG_NAME = "catalog_name"; static final String TABLE_NAME = "table_name"; static final String TABLE_NAMESPACE = "table_namespace"; static final String RECORD_TYPE = "iceberg_type"; static final String TABLE_RECORD_TYPE = "TABLE"; static final String VIEW_RECORD_TYPE = "VIEW"; ``` ########## crates/catalog/sql/Cargo.toml: ########## @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "iceberg-catalog-sql" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +rust-version = { workspace = true } + +categories = ["database"] +description = "Apache Iceberg Rust Sql Catalog" +repository = { workspace = true } +license = { workspace = true } +keywords = ["iceberg", "sql", "catalog"] + +[dependencies] +anyhow = "1" Review Comment: anyhow = { workspace = true } ########## crates/catalog/sql/src/catalog.rs: ########## @@ -0,0 +1,397 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use dashmap::DashMap; +use futures::{lock::Mutex, AsyncReadExt, AsyncWriteExt}; +use sqlx::{ + any::{install_default_drivers, AnyConnectOptions, AnyRow}, + AnyConnection, ConnectOptions, Connection, Row, +}; +use std::collections::HashMap; + +use iceberg::{ + io::FileIO, spec::TableMetadata, table::Table, Catalog, Error, ErrorKind, Namespace, + NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, +}; +use uuid::Uuid; + +use crate::error::from_sqlx_error; + +#[derive(Debug)] +/// Sql catalog implementation. +pub struct SqlCatalog { + name: String, + connection: Arc<Mutex<AnyConnection>>, + storage: FileIO, + cache: Arc<DashMap<TableIdent, (String, TableMetadata)>>, +} + +impl SqlCatalog { + /// Create new sql catalog instance + pub async fn new(url: &str, name: &str, storage: FileIO) -> Result<Self> { + install_default_drivers(); + + let mut connection = AnyConnectOptions::connect( + &AnyConnectOptions::from_url(&url.try_into()?).map_err(from_sqlx_error)?, + ) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_tables ( + catalog_name text not null, + table_namespace text not null, + table_name text not null, + metadata_location text not null, + previous_metadata_location text, + primary key (catalog_name, table_namespace, table_name) + );", + ) + .execute(&mut **txn) + .await + }) + }) + .await + .map_err(from_sqlx_error)?; + + connection + .transaction(|txn| { + Box::pin(async move { + sqlx::query( + "create table if not exists iceberg_namespace_properties ( + catalog_name text not null, Review Comment: `VARCHAR(255)` here. Maybe we can copy&paste SQL from [java iceberg](https://github.com/apache/iceberg/blob/2519ab43d654927802cc02e19c917ce90e8e0265/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L370-L389)? ```java + CATALOG_NAME + " VARCHAR(255) NOT NULL," + NAMESPACE_NAME + " VARCHAR(255) NOT NULL," + NAMESPACE_PROPERTY_KEY + " VARCHAR(255)," + NAMESPACE_PROPERTY_VALUE + " VARCHAR(1000)," ``` ########## crates/catalog/sql/Cargo.toml: ########## @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "iceberg-catalog-sql" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +rust-version = { workspace = true } + +categories = ["database"] +description = "Apache Iceberg Rust Sql Catalog" +repository = { workspace = true } +license = { workspace = true } +keywords = ["iceberg", "sql", "catalog"] + +[dependencies] +anyhow = "1" +async-trait = { workspace = true } +chrono = { workspace = true } +dashmap = "5.5.3" +futures = { workspace = true } +iceberg = { workspace = true } +log = "0.4.20" +opendal = { workspace = true } +serde = { workspace = true } +serde_derive = { workspace = true } +serde_json = { workspace = true } +sqlx = { version = "0.7.2", features = ["runtime-tokio-rustls", "any", "sqlite", "postgres", "mysql"], default-features = false } Review Comment: cc @liurenjie1024 @Xuanwo @Fokko -- 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: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org