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

feat: support create partition table for non REST catalog #577

Closed
wants to merge 5 commits into from
Closed
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
52 changes: 24 additions & 28 deletions crates/catalog/rest/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ use std::str::FromStr;

use async_trait::async_trait;
use iceberg::io::FileIO;
use iceberg::spec::UnboundPartitionSpec;
use iceberg::table::Table;
use iceberg::{
Catalog, Error, ErrorKind, Namespace, NamespaceIdent, Result, TableCommit, TableCreation,
Expand Down Expand Up @@ -498,7 +499,7 @@ impl Catalog for RestCatalog {
name: creation.name,
location: creation.location,
schema: creation.schema,
partition_spec: creation.partition_spec,
partition_spec: creation.partition_spec.map(UnboundPartitionSpec::from),
write_order: creation.sort_order,
// We don't support stage create yet.
stage_create: Some(false),
Expand Down Expand Up @@ -700,9 +701,9 @@ mod tests {

use chrono::{TimeZone, Utc};
use iceberg::spec::{
FormatVersion, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot,
SnapshotLog, SortDirection, SortField, SortOrder, Summary, Transform, Type,
UnboundPartitionField, UnboundPartitionSpec,
FormatVersion, NestedField, NullOrder, Operation, PartitionSpecBuilder, PrimitiveType,
Schema, Snapshot, SnapshotLog, SortDirection, SortField, SortOrder, Summary, Transform,
Type,
};
use iceberg::transaction::Transaction;
use mockito::{Mock, Server, ServerGuard};
Expand Down Expand Up @@ -1470,33 +1471,28 @@ mod tests {

let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build());

let schema = Schema::builder()
.with_fields(vec![
NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(),
NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(),
NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(),
])
.with_schema_id(1)
.with_identifier_field_ids(vec![2])
.build()
.unwrap();

let partition_spec = PartitionSpecBuilder::new(&schema)
.add_partition_field("foo", "id", Transform::Truncate(3))
.unwrap()
.build()
.unwrap();

let table_creation = TableCreation::builder()
.name("test1".to_string())
.schema(
Schema::builder()
.with_fields(vec![
NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String))
.into(),
NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(),
NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean))
.into(),
])
.with_schema_id(1)
.with_identifier_field_ids(vec![2])
.build()
.unwrap(),
)
.schema(schema)
.properties(HashMap::from([("owner".to_string(), "testx".to_string())]))
.partition_spec(
UnboundPartitionSpec::builder()
.add_partition_fields(vec![UnboundPartitionField::builder()
.source_id(1)
.transform(Transform::Truncate(3))
.name("id".to_string())
.build()])
.unwrap()
.build(),
)
.partition_spec(partition_spec)
.sort_order(
SortOrder::builder()
.with_sort_field(
Expand Down
5 changes: 5 additions & 0 deletions crates/examples/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ license = { workspace = true }
rust-version = { workspace = true }

[dependencies]
futures = { workspace = true }
iceberg = { workspace = true }
iceberg-catalog-rest = { workspace = true }
tokio = { version = "1", features = ["full"] }
Expand All @@ -36,3 +37,7 @@ path = "src/rest_catalog_namespace.rs"
[[example]]
name = "rest-catalog-table"
path = "src/rest_catalog_table.rs"

[[example]]
name = "rest-catalog-read-data"
path = "src/rest_catalog_read_data.rs"
12 changes: 4 additions & 8 deletions crates/examples/src/rest_catalog_namespace.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,14 @@
use std::collections::HashMap;

use iceberg::{Catalog, NamespaceIdent};
use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig};
use utils::get_rest_catalog;
mod utils;

#[tokio::main]
async fn main() {
// ANCHOR: create_catalog
// Create catalog
let config = RestCatalogConfig::builder()
.uri("http://localhost:8080".to_string())
.build();

let catalog = RestCatalog::new(config);
let catalog = get_rest_catalog();
// ANCHOR_END: create_catalog

// ANCHOR: list_all_namespace
Expand All @@ -38,8 +35,7 @@ async fn main() {
// ANCHOR_END: list_all_namespace

// ANCHOR: create_namespace
let namespace_id =
NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap();
let namespace_id = NamespaceIdent::from_vec(vec!["ns1".to_string()]).unwrap();
// Create namespace
let ns = catalog
.create_namespace(
Expand Down
42 changes: 42 additions & 0 deletions crates/examples/src/rest_catalog_read_data.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
// 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 futures::TryStreamExt;
use iceberg::{Catalog, Result, TableIdent};
mod utils;

#[tokio::main]
async fn main() -> Result<()> {
let catalog = utils::get_rest_catalog();

// There should be a table `ns1.t1` pre-created with column name `a`,`b`,`c`
let table_id = TableIdent::from_strs(["ns1", "t1"]).unwrap();
let table = catalog.load_table(&table_id).await?;

// Build table scan.
let stream = table
.scan()
.select(["a", "b", "c"])
.build()?
.to_arrow()
.await?;

// Consume this stream like arrow record batch stream.
let data: Vec<_> = stream.try_collect().await?;
println!("data: {:?}", data);
Ok(())
}
55 changes: 28 additions & 27 deletions crates/examples/src/rest_catalog_table.rs
Copy link
Contributor

Choose a reason for hiding this comment

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

Some suggestions:

  1. Please don't remove ANCHOR comments, those are for websites.
  2. It would be better to add another create partitioned table example rather modifying current one.

Original file line number Diff line number Diff line change
Expand Up @@ -17,53 +17,54 @@

use std::collections::HashMap;

use iceberg::spec::{NestedField, PrimitiveType, Schema, Type};
use iceberg::spec::{NestedField, PartitionSpecBuilder, PrimitiveType, Schema, Transform, Type};
use iceberg::{Catalog, TableCreation, TableIdent};
use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig};
use iceberg_catalog_rest::RestCatalog;

#[tokio::main]
async fn main() {
// Create catalog
let config = RestCatalogConfig::builder()
.uri("http://localhost:8080".to_string())
.build();

let catalog = RestCatalog::new(config);

// ANCHOR: create_table
let table_id = TableIdent::from_strs(["default", "t1"]).unwrap();
mod utils;

fn get_table_creation(table_id: &TableIdent) -> TableCreation {
let table_schema = Schema::builder()
.with_fields(vec![
NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(),
NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(),
NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(),
NestedField::optional(1, "a", Type::Primitive(PrimitiveType::String)).into(),
NestedField::required(2, "b", Type::Primitive(PrimitiveType::Int)).into(),
NestedField::optional(3, "c", Type::Primitive(PrimitiveType::Boolean)).into(),
])
.with_schema_id(1)
.with_identifier_field_ids(vec![2])
.build()
.unwrap();

// Create table
let table_creation = TableCreation::builder()
let p = PartitionSpecBuilder::new(&table_schema)
.add_partition_field("a", "bucket_a", Transform::Bucket(16))
.unwrap()
.add_partition_field("b", "b", Transform::Identity)
.unwrap()
.build()
.unwrap();

TableCreation::builder()
.name(table_id.name.clone())
.schema(table_schema.clone())
.partition_spec(p)
.properties(HashMap::from([("owner".to_string(), "testx".to_string())]))
.build();
.build()
}

#[tokio::main]
async fn main() {
let catalog: RestCatalog = utils::get_rest_catalog();

let table_id = TableIdent::from_strs(["ns1", "t1"]).unwrap();
catalog.drop_table(&table_id).await.unwrap();

let table_creation = get_table_creation(&table_id);
let table = catalog
.create_table(&table_id.namespace, table_creation)
.await
.unwrap();

println!("Table created: {:?}", table.metadata());
// ANCHOR_END: create_table

// ANCHOR: load_table
let table2 = catalog
.load_table(&TableIdent::from_strs(["default", "t2"]).unwrap())
.await
.unwrap();
let table2 = catalog.load_table(&table_id).await.unwrap();
println!("{:?}", table2.metadata());
// ANCHOR_END: load_table
}
31 changes: 31 additions & 0 deletions crates/examples/src/utils.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// 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::env;

use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig};

fn get_catalog_uri_from_env() -> String {
env::var("CATALOG_URI").unwrap_or("http://localhost:8080".to_string())
}

pub fn get_rest_catalog() -> RestCatalog {
let config = RestCatalogConfig::builder()
.uri(get_catalog_uri_from_env())
.build();
RestCatalog::new(config)
}
6 changes: 3 additions & 3 deletions crates/iceberg/src/catalog/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,8 @@ use typed_builder::TypedBuilder;
use uuid::Uuid;

use crate::spec::{
FormatVersion, Schema, Snapshot, SnapshotReference, SortOrder, TableMetadataBuilder,
UnboundPartitionSpec, ViewRepresentations,
FormatVersion, PartitionSpec, Schema, Snapshot, SnapshotReference, SortOrder,
TableMetadataBuilder, UnboundPartitionSpec, ViewRepresentations,
};
use crate::table::Table;
use crate::{Error, ErrorKind, Result};
Expand Down Expand Up @@ -230,7 +230,7 @@ pub struct TableCreation {
pub schema: Schema,
/// The partition spec of the table, could be None.
#[builder(default, setter(strip_option, into))]
pub partition_spec: Option<UnboundPartitionSpec>,
pub partition_spec: Option<PartitionSpec>,
/// The sort order of the table.
#[builder(default, setter(strip_option))]
pub sort_order: Option<SortOrder>,
Expand Down
8 changes: 2 additions & 6 deletions crates/iceberg/src/spec/table_metadata.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
use std::cmp::Ordering;
use std::collections::HashMap;
use std::fmt::{Display, Formatter};
use std::hash::Hash;
use std::sync::Arc;

use _serde::TableMetadataEnum;
Expand Down Expand Up @@ -301,12 +302,7 @@ impl TableMetadataBuilder {
} = table_creation;

let partition_specs = match partition_spec {
Some(_) => {
return Err(Error::new(
ErrorKind::FeatureUnsupported,
"Can't create table with partition spec now",
))
}
Some(p) => HashMap::from([(p.spec_id(), Arc::new(p))]),
Copy link
Contributor

Choose a reason for hiding this comment

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

This is incorrect, we can't simply add a spec. We should use AddPartitionSpec transaction api.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

sorry, I couldn't understand AddPartitionSpec transaction api, could you provide more context?

Copy link
Contributor

Choose a reason for hiding this comment

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

You can refer to iceberg-python's implementation The whole table creation process is treated as a transaction, similar to transactions.

None => HashMap::from([(
DEFAULT_PARTITION_SPEC_ID,
Arc::new(PartitionSpec {
Expand Down
Loading
Loading