Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Improve object store builder #63

Merged
merged 6 commits into from
Nov 18, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
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
7 changes: 3 additions & 4 deletions datafusion_iceberg/examples/dataframe.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,23 +3,22 @@ use datafusion::arrow::record_batch::RecordBatch;
use datafusion::dataframe::DataFrameWriteOptions;
use datafusion::prelude::SessionContext;
use datafusion_iceberg::catalog::catalog::IcebergCatalog;
use iceberg_rust::catalog::bucket::ObjectStoreBuilder;
use iceberg_rust::catalog::identifier::Identifier;
use iceberg_rust::catalog::Catalog;
use iceberg_rust::spec::schema::Schema;
use iceberg_rust::spec::types::{StructField, StructType};
use iceberg_rust::table::Table;
use iceberg_sql_catalog::SqlCatalog;
use object_store::memory::InMemory;
use object_store::ObjectStore;

use std::sync::Arc;

#[tokio::main]
pub(crate) async fn main() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "test", object_store.clone())
SqlCatalog::new("sqlite://", "test", object_store)
.await
.unwrap(),
);
Expand Down
6 changes: 3 additions & 3 deletions datafusion_iceberg/examples/insert_csv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,8 @@ use datafusion::{
execution::{context::SessionContext, SessionStateBuilder},
};
use datafusion_expr::ScalarUDF;
use iceberg_rust::catalog::bucket::ObjectStoreBuilder;
use iceberg_sql_catalog::SqlCatalogList;
use object_store::{memory::InMemory, ObjectStore};

use datafusion_iceberg::{
catalog::catalog_list::IcebergCatalogList,
Expand All @@ -16,9 +16,9 @@ use datafusion_iceberg::{

#[tokio::main]
async fn main() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();
let iceberg_catalog_list = Arc::new(
SqlCatalogList::new("sqlite://", object_store.clone())
SqlCatalogList::new("sqlite://", object_store)
.await
.unwrap(),
);
Expand Down
8 changes: 3 additions & 5 deletions datafusion_iceberg/examples/insert_table.rs
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
use datafusion::{arrow::array::Int64Array, prelude::SessionContext};
use datafusion_iceberg::DataFusionTable;
use iceberg_rust::{
catalog::Catalog,
catalog::{bucket::ObjectStoreBuilder, Catalog},
spec::{
partition::{PartitionField, PartitionSpec, Transform},
schema::Schema,
Expand All @@ -10,17 +10,15 @@ use iceberg_rust::{
table::Table,
};
use iceberg_sql_catalog::SqlCatalog;
use object_store::memory::InMemory;
use object_store::ObjectStore;

use std::sync::Arc;

#[tokio::main]
pub(crate) async fn main() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "test", object_store.clone())
SqlCatalog::new("sqlite://", "test", object_store)
.await
.unwrap(),
);
Expand Down
7 changes: 3 additions & 4 deletions datafusion_iceberg/examples/refresh_materialized_view.rs
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
use datafusion::{arrow::array::Int64Array, prelude::SessionContext};
use datafusion_iceberg::catalog::catalog::IcebergCatalog;
use datafusion_iceberg::materialized_view::refresh_materialized_view;
use iceberg_rust::catalog::bucket::ObjectStoreBuilder;
use iceberg_rust::catalog::CatalogList;
use iceberg_rust::materialized_view::MaterializedView;
use iceberg_rust::spec::partition::PartitionSpec;
Expand All @@ -12,16 +13,14 @@ use iceberg_rust::spec::{
};
use iceberg_rust::table::Table;
use iceberg_sql_catalog::SqlCatalogList;
use object_store::memory::InMemory;
use object_store::ObjectStore;

use std::sync::Arc;
#[tokio::main]
pub(crate) async fn main() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog_list = Arc::new(
SqlCatalogList::new("sqlite://", object_store.clone())
SqlCatalogList::new("sqlite://", object_store)
.await
.unwrap(),
);
Expand Down
16 changes: 9 additions & 7 deletions datafusion_iceberg/src/materialized_view.rs
Original file line number Diff line number Diff line change
Expand Up @@ -210,10 +210,13 @@ pub async fn refresh_materialized_view(
mod tests {

use datafusion::{arrow::array::Int64Array, prelude::SessionContext};
use iceberg_rust::spec::{
partition::{PartitionField, Transform},
schema::Schema,
types::{PrimitiveType, StructField, StructType, Type},
use iceberg_rust::{
catalog::bucket::ObjectStoreBuilder,
spec::{
partition::{PartitionField, Transform},
schema::Schema,
types::{PrimitiveType, StructField, StructType, Type},
},
};
use iceberg_rust::{
catalog::CatalogList,
Expand All @@ -225,17 +228,16 @@ mod tests {
table::Table,
};
use iceberg_sql_catalog::SqlCatalogList;
use object_store::{memory::InMemory, ObjectStore};
use std::sync::Arc;

use crate::{catalog::catalog::IcebergCatalog, materialized_view::refresh_materialized_view};

#[tokio::test]
pub async fn test_datafusion_refresh_materialized_view() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog_list = Arc::new(
SqlCatalogList::new("sqlite://", object_store.clone())
SqlCatalogList::new("sqlite://", object_store)
.await
.unwrap(),
);
Expand Down
6 changes: 3 additions & 3 deletions datafusion_iceberg/src/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -506,8 +506,8 @@ mod tests {
execution::{context::SessionContext, SessionStateBuilder},
};
use datafusion_expr::ScalarUDF;
use iceberg_rust::catalog::bucket::ObjectStoreBuilder;
use iceberg_sql_catalog::SqlCatalogList;
use object_store::{memory::InMemory, ObjectStore};
use tokio::time::sleep;

use crate::{
Expand All @@ -517,9 +517,9 @@ mod tests {

#[tokio::test]
async fn test_planner() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();
let iceberg_catalog_list = Arc::new(
SqlCatalogList::new("sqlite://", object_store.clone())
SqlCatalogList::new("sqlite://", object_store)
.await
.unwrap(),
);
Expand Down
34 changes: 17 additions & 17 deletions datafusion_iceberg/src/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -575,14 +575,6 @@ impl DataSink for IcebergDataSink {
mod tests {

use datafusion::{arrow::array::Int64Array, prelude::SessionContext};
use iceberg_rust::{
catalog::tabular::Tabular,
spec::{
partition::{PartitionField, Transform},
schema::Schema,
types::{PrimitiveType, StructField, StructType, Type},
},
};
use iceberg_rust::{
catalog::Catalog,
spec::{
Expand All @@ -592,18 +584,26 @@ mod tests {
table::Table,
view::View,
};
use iceberg_rust::{
catalog::{bucket::ObjectStoreBuilder, tabular::Tabular},
spec::{
partition::{PartitionField, Transform},
schema::Schema,
types::{PrimitiveType, StructField, StructType, Type},
},
};
use iceberg_sql_catalog::SqlCatalog;
use object_store::{memory::InMemory, ObjectStore};

use std::{ops::Deref, sync::Arc};

use crate::{catalog::catalog::IcebergCatalog, DataFusionTable};

#[tokio::test]
pub async fn test_datafusion_table_insert() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "test", object_store.clone())
SqlCatalog::new("sqlite://", "test", object_store)
.await
.unwrap(),
);
Expand Down Expand Up @@ -774,10 +774,10 @@ mod tests {

#[tokio::test]
pub async fn test_datafusion_table_insert_partitioned() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "test", object_store.clone())
SqlCatalog::new("sqlite://", "test", object_store)
.await
.unwrap(),
);
Expand Down Expand Up @@ -965,10 +965,10 @@ mod tests {

#[tokio::test]
pub async fn test_datafusion_table_branch_insert() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "iceberg", object_store.clone())
SqlCatalog::new("sqlite://", "iceberg", object_store)
.await
.unwrap(),
);
Expand Down Expand Up @@ -1145,10 +1145,10 @@ mod tests {

#[tokio::test]
pub async fn test_datafusion_view_scan() {
let object_store: Arc<dyn ObjectStore> = Arc::new(InMemory::new());
let object_store = ObjectStoreBuilder::memory();

let catalog: Arc<dyn Catalog> = Arc::new(
SqlCatalog::new("sqlite://", "test", object_store.clone())
SqlCatalog::new("sqlite://", "test", object_store)
.await
.unwrap(),
);
Expand Down
23 changes: 10 additions & 13 deletions datafusion_iceberg/tests/integration_trino.rs
Original file line number Diff line number Diff line change
Expand Up @@ -183,14 +183,15 @@ async fn integration_trino_rest() {
.await
.unwrap();

let object_store = ObjectStoreBuilder::S3(
AmazonS3Builder::new()
.with_region("us-east-1")
.with_allow_http(true)
.with_access_key_id("user")
.with_secret_access_key("password")
.with_endpoint(format!("http://{}:{}", localstack_host, localstack_port)),
);
let object_store = ObjectStoreBuilder::aws()
.with_config("aws_access_key_id".parse().unwrap(), "user")
.with_config("aws_secret_access_key".parse().unwrap(), "password")
.with_config(
"endpoint".parse().unwrap(),
format!("http://{}:{}", localstack_host, localstack_port),
)
.with_config("region".parse().unwrap(), "us-east-1")
.with_config("allow_http".parse().unwrap(), "true");

let catalog = Arc::new(RestCatalog::new(
None,
Expand Down Expand Up @@ -358,11 +359,7 @@ async fn integration_trino_sql() {
postgres_host, postgres_port
),
"iceberg",
Arc::new(
object_store
.build(iceberg_rust::catalog::bucket::Bucket::S3("warehouse"))
.unwrap(),
),
object_store,
)
.await
.unwrap(),
Expand Down
Loading
Loading