From 5844876ded34fe80d9fa45bb2b9a1c1d742f8cd7 Mon Sep 17 00:00:00 2001 From: zhouzilong <529620861@qq.com> Date: Wed, 21 Feb 2024 16:53:44 -0500 Subject: [PATCH 1/4] import iceberg --- .gitignore | 4 +- Cargo.toml | 74 +- Makefile | 50 + crates/examples/Cargo.toml | 37 + crates/examples/README.md | 21 + crates/examples/src/rest_catalog_namespace.rs | 53 + crates/examples/src/rest_catalog_table.rs | 68 + crates/iceberg/Cargo.toml | 67 + crates/iceberg/DEPENDENCIES.rust.tsv | 276 ++ crates/iceberg/src/avro/mod.rs | 21 + crates/iceberg/src/avro/schema.rs | 955 +++++++ crates/iceberg/src/catalog/mod.rs | 1068 ++++++++ crates/iceberg/src/error.rs | 429 ++++ crates/iceberg/src/expr/mod.rs | 42 + crates/iceberg/src/expr/predicate.rs | 93 + crates/iceberg/src/expr/term.rs | 37 + crates/iceberg/src/io.rs | 508 ++++ crates/iceberg/src/lib.rs | 55 + crates/iceberg/src/scan.rs | 448 ++++ crates/iceberg/src/spec/datatypes.rs | 1056 ++++++++ crates/iceberg/src/spec/manifest.rs | 2007 +++++++++++++++ crates/iceberg/src/spec/manifest_list.rs | 1468 +++++++++++ crates/iceberg/src/spec/mod.rs | 40 + crates/iceberg/src/spec/partition.rs | 492 ++++ crates/iceberg/src/spec/schema.rs | 1289 ++++++++++ crates/iceberg/src/spec/snapshot.rs | 404 +++ crates/iceberg/src/spec/sort.rs | 480 ++++ crates/iceberg/src/spec/table_metadata.rs | 1572 ++++++++++++ crates/iceberg/src/spec/transform.rs | 861 +++++++ crates/iceberg/src/spec/values.rs | 2237 +++++++++++++++++ crates/iceberg/src/table.rs | 65 + crates/iceberg/src/transaction.rs | 370 +++ crates/iceberg/src/transform/bucket.rs | 245 ++ crates/iceberg/src/transform/identity.rs | 31 + crates/iceberg/src/transform/mod.rs | 55 + crates/iceberg/src/transform/temporal.rs | 412 +++ crates/iceberg/src/transform/truncate.rs | 218 ++ crates/iceberg/src/transform/void.rs | 30 + crates/iceberg/src/writer/file_writer/mod.rs | 39 + crates/iceberg/src/writer/mod.rs | 35 + .../testdata/avro_schema_manifest_entry.json | 286 +++ .../avro_schema_manifest_file_v1.json | 139 + .../avro_schema_manifest_file_v2.json | 141 ++ .../testdata/example_table_metadata_v2.json | 61 + .../TableMetadataUnsupportedVersion.json | 36 + .../table_metadata/TableMetadataV1Valid.json | 42 + .../TableMetadataV2CurrentSchemaNotFound.json | 88 + ...TableMetadataV2MissingLastPartitionId.json | 73 + .../TableMetadataV2MissingPartitionSpecs.json | 67 + .../TableMetadataV2MissingSchemas.json | 71 + .../TableMetadataV2MissingSortOrder.json | 54 + .../table_metadata/TableMetadataV2Valid.json | 122 + .../TableMetadataV2ValidMinimal.json | 71 + crates/rest/Cargo.toml | 48 + crates/rest/DEPENDENCIES.rust.tsv | 295 +++ crates/rest/src/catalog.rs | 1604 ++++++++++++ crates/rest/src/lib.rs | 23 + .../rest/testdata/create_table_response.json | 53 + crates/rest/testdata/load_table_response.json | 68 + .../testdata/rest_catalog/docker-compose.yaml | 65 + .../rest/testdata/update_table_response.json | 40 + crates/rest/tests/rest_catalog_test.rs | 376 +++ crates/test_utils/Cargo.toml | 31 + crates/test_utils/src/cmd.rs | 41 + crates/test_utils/src/docker.rs | 102 + crates/test_utils/src/lib.rs | 41 + 66 files changed, 21744 insertions(+), 6 deletions(-) create mode 100644 Makefile create mode 100644 crates/examples/Cargo.toml create mode 100644 crates/examples/README.md create mode 100644 crates/examples/src/rest_catalog_namespace.rs create mode 100644 crates/examples/src/rest_catalog_table.rs create mode 100644 crates/iceberg/Cargo.toml create mode 100644 crates/iceberg/DEPENDENCIES.rust.tsv create mode 100644 crates/iceberg/src/avro/mod.rs create mode 100644 crates/iceberg/src/avro/schema.rs create mode 100644 crates/iceberg/src/catalog/mod.rs create mode 100644 crates/iceberg/src/error.rs create mode 100644 crates/iceberg/src/expr/mod.rs create mode 100644 crates/iceberg/src/expr/predicate.rs create mode 100644 crates/iceberg/src/expr/term.rs create mode 100644 crates/iceberg/src/io.rs create mode 100644 crates/iceberg/src/lib.rs create mode 100644 crates/iceberg/src/scan.rs create mode 100644 crates/iceberg/src/spec/datatypes.rs create mode 100644 crates/iceberg/src/spec/manifest.rs create mode 100644 crates/iceberg/src/spec/manifest_list.rs create mode 100644 crates/iceberg/src/spec/mod.rs create mode 100644 crates/iceberg/src/spec/partition.rs create mode 100644 crates/iceberg/src/spec/schema.rs create mode 100644 crates/iceberg/src/spec/snapshot.rs create mode 100644 crates/iceberg/src/spec/sort.rs create mode 100644 crates/iceberg/src/spec/table_metadata.rs create mode 100644 crates/iceberg/src/spec/transform.rs create mode 100644 crates/iceberg/src/spec/values.rs create mode 100644 crates/iceberg/src/table.rs create mode 100644 crates/iceberg/src/transaction.rs create mode 100644 crates/iceberg/src/transform/bucket.rs create mode 100644 crates/iceberg/src/transform/identity.rs create mode 100644 crates/iceberg/src/transform/mod.rs create mode 100644 crates/iceberg/src/transform/temporal.rs create mode 100644 crates/iceberg/src/transform/truncate.rs create mode 100644 crates/iceberg/src/transform/void.rs create mode 100644 crates/iceberg/src/writer/file_writer/mod.rs create mode 100644 crates/iceberg/src/writer/mod.rs create mode 100644 crates/iceberg/testdata/avro_schema_manifest_entry.json create mode 100644 crates/iceberg/testdata/avro_schema_manifest_file_v1.json create mode 100644 crates/iceberg/testdata/avro_schema_manifest_file_v2.json create mode 100644 crates/iceberg/testdata/example_table_metadata_v2.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json create mode 100644 crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json create mode 100644 crates/rest/Cargo.toml create mode 100644 crates/rest/DEPENDENCIES.rust.tsv create mode 100644 crates/rest/src/catalog.rs create mode 100644 crates/rest/src/lib.rs create mode 100644 crates/rest/testdata/create_table_response.json create mode 100644 crates/rest/testdata/load_table_response.json create mode 100644 crates/rest/testdata/rest_catalog/docker-compose.yaml create mode 100644 crates/rest/testdata/update_table_response.json create mode 100644 crates/rest/tests/rest_catalog_test.rs create mode 100644 crates/test_utils/Cargo.toml create mode 100644 crates/test_utils/src/cmd.rs create mode 100644 crates/test_utils/src/docker.rs create mode 100644 crates/test_utils/src/lib.rs diff --git a/.gitignore b/.gitignore index 0000196..6d586ef 100644 --- a/.gitignore +++ b/.gitignore @@ -14,4 +14,6 @@ Cargo.lock *.pdb # macOS resource forks and .DS_Store files -.DS_Store \ No newline at end of file +.DS_Store + +.vscode \ No newline at end of file diff --git a/Cargo.toml b/Cargo.toml index 380993f..282d0a2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,8 +1,72 @@ -[package] -name = "catalog2" -version = "0.1.0" +# 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. + +[workspace] +resolver = "2" +members = ["crates/rest", "crates/examples", "crates/iceberg", "crates/test_utils"] + +[workspace.package] +version = "0.2.0" edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html +license = "Apache-2.0" +repository = "https://github.com/cmu-db/15721-s24-catalog2" +rust-version = "1.75.0" -[dependencies] +[workspace.dependencies] +anyhow = "1.0.72" +apache-avro = "0.16" +arrow-arith = { version = ">=46" } +arrow-array = { version = ">=46" } +arrow-schema = { version = ">=46" } +async-trait = "0.1" +bimap = "0.6" +bitvec = "1.0.1" +chrono = "0.4" +derive_builder = "0.13.0" +either = "1" +env_logger = "0.11.0" +futures = "0.3" +iceberg = { path = "./crates/iceberg" } +iceberg-catalog-rest = { path = "./crates/rest" } +itertools = "0.12" +lazy_static = "1" +log = "^0.4" +mockito = "^1" +murmur3 = "0.5.2" +once_cell = "1" +opendal = "0.45" +ordered-float = "4.0.0" +pretty_assertions = "1.4.0" +port_scanner = "0.1.5" +reqwest = { version = "^0.11", features = ["json"] } +rust_decimal = "1.31.0" +serde = { version = "^1.0", features = ["rc"] } +serde_bytes = "0.11.8" +serde_derive = "^1.0" +serde_json = "^1.0" +serde_repr = "0.1.16" +serde_with = "3.4.0" +tempfile = "3.8" +tokio = { version = "1", features = ["macros"] } +typed-builder = "^0.18" +url = "2" +urlencoding = "2" +uuid = "1.6.1" +volo-thrift = "0.9.2" +hive_metastore = "0.0.2" +tera = "1" diff --git a/Makefile b/Makefile new file mode 100644 index 0000000..d411cce --- /dev/null +++ b/Makefile @@ -0,0 +1,50 @@ +# 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. + +.EXPORT_ALL_VARIABLES: + +RUST_LOG = debug + +build: + cargo build + +check-fmt: + cargo fmt --all -- --check + +check-clippy: + cargo clippy --all-targets --all-features --workspace -- -D warnings + +cargo-sort: + cargo install cargo-sort + cargo sort -c -w + +fix-toml: + cargo install taplo-cli --locked + taplo fmt + +check-toml: + cargo install taplo-cli --locked + taplo check + +check: check-fmt check-clippy cargo-sort check-toml + +unit-test: + cargo test --no-fail-fast --lib --all-features --workspace + +test: + cargo test --no-fail-fast --all-targets --all-features --workspace + cargo test --no-fail-fast --doc --all-features --workspace \ No newline at end of file diff --git a/crates/examples/Cargo.toml b/crates/examples/Cargo.toml new file mode 100644 index 0000000..57596c2 --- /dev/null +++ b/crates/examples/Cargo.toml @@ -0,0 +1,37 @@ +# 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-examples" +version = { workspace = true } +edition = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +rust-version = { workspace = true } + +[dependencies] +iceberg = { workspace = true } +iceberg-catalog-rest = { workspace = true } +tokio = { version = "1", features = ["full"] } + +[[example]] +name = "rest-catalog-namespace" +path = "src/rest_catalog_namespace.rs" + +[[example]] +name = "rest-catalog-table" +path = "src/rest_catalog_table.rs" diff --git a/crates/examples/README.md b/crates/examples/README.md new file mode 100644 index 0000000..335d2ea --- /dev/null +++ b/crates/examples/README.md @@ -0,0 +1,21 @@ + + +Example usage codes for `iceberg-rust`. Currently, these examples can't run directly since it requires setting up of +environments for catalogs, for example, rest catalog server. \ No newline at end of file diff --git a/crates/examples/src/rest_catalog_namespace.rs b/crates/examples/src/rest_catalog_namespace.rs new file mode 100644 index 0000000..0a3b00b --- /dev/null +++ b/crates/examples/src/rest_catalog_namespace.rs @@ -0,0 +1,53 @@ +// 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 iceberg::{Catalog, NamespaceIdent}; +use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; +use std::collections::HashMap; + +#[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).await.unwrap(); + // ANCHOR_END: create_catalog + + // ANCHOR: list_all_namespace + // List all namespaces + let all_namespaces = catalog.list_namespaces(None).await.unwrap(); + println!("Namespaces in current catalog: {:?}", all_namespaces); + // ANCHOR_END: list_all_namespace + + // ANCHOR: create_namespace + let namespace_id = + NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(); + // Create namespace + let ns = catalog + .create_namespace( + &namespace_id, + HashMap::from([("key1".to_string(), "value1".to_string())]), + ) + .await + .unwrap(); + + println!("Namespace created: {:?}", ns); + // ANCHOR_END: create_namespace +} diff --git a/crates/examples/src/rest_catalog_table.rs b/crates/examples/src/rest_catalog_table.rs new file mode 100644 index 0000000..9fb3dd7 --- /dev/null +++ b/crates/examples/src/rest_catalog_table.rs @@ -0,0 +1,68 @@ +// 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 iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; +use iceberg::{Catalog, TableCreation, TableIdent}; +use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; +use std::collections::HashMap; + +#[tokio::main] +async fn main() { + // Create catalog + let config = RestCatalogConfig::builder() + .uri("http://localhost:8080".to_string()) + .build(); + + let catalog = RestCatalog::new(config).await.unwrap(); + + // ANCHOR: create_table + let table_id = TableIdent::from_strs(["default", "t1"]).unwrap(); + + 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(), + ]) + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .build() + .unwrap(); + + // Create table + let table_creation = TableCreation::builder() + .name(table_id.name.clone()) + .schema(table_schema.clone()) + .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) + .build(); + + 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(); + println!("{:?}", table2.metadata()); + // ANCHOR_END: load_table +} diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml new file mode 100644 index 0000000..694e12e --- /dev/null +++ b/crates/iceberg/Cargo.toml @@ -0,0 +1,67 @@ +# 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" +version = { workspace = true } +edition = { workspace = true } +rust-version = { workspace = true } + +categories = ["database"] +description = "Apache Iceberg Rust implementation" +repository = { workspace = true } +license = { workspace = true } +keywords = ["iceberg"] + +[dependencies] +anyhow = { workspace = true } +apache-avro = { workspace = true } +arrow-arith = { workspace = true } +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +async-trait = { workspace = true } +bimap = { workspace = true } +bitvec = { workspace = true } +chrono = { workspace = true } +derive_builder = { workspace = true } +either = { workspace = true } +futures = { workspace = true } +itertools = { workspace = true } +lazy_static = { workspace = true } +log = { workspace = true } +murmur3 = { workspace = true } +once_cell = { workspace = true } +opendal = { workspace = true } +ordered-float = { workspace = true } +reqwest = { workspace = true } +rust_decimal = { workspace = true } +serde = { workspace = true } +serde_bytes = { workspace = true } +serde_derive = { workspace = true } +serde_json = { workspace = true } +serde_repr = { workspace = true } +serde_with = { workspace = true } +typed-builder = { workspace = true } +url = { workspace = true } +urlencoding = { workspace = true } +uuid = { workspace = true } + +[dev-dependencies] +pretty_assertions = { workspace = true } +tempfile = { workspace = true } +tera = { workspace = true } +tokio = { workspace = true } diff --git a/crates/iceberg/DEPENDENCIES.rust.tsv b/crates/iceberg/DEPENDENCIES.rust.tsv new file mode 100644 index 0000000..f47f941 --- /dev/null +++ b/crates/iceberg/DEPENDENCIES.rust.tsv @@ -0,0 +1,276 @@ +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT OpenSSL Unicode-DFS-2016 Unlicense Zlib +addr2line@0.21.0 X X +adler@1.0.2 X X X +adler32@1.2.0 X +ahash@0.8.6 X X +android-tzdata@0.1.1 X X +android_system_properties@0.1.5 X X +anyhow@1.0.77 X X +apache-avro@0.16.0 X +arrayvec@0.7.4 X X +arrow-arith@49.0.0 X +arrow-array@49.0.0 X +arrow-buffer@49.0.0 X +arrow-data@49.0.0 X +arrow-schema@49.0.0 X +async-compat@0.2.3 X X +async-trait@0.1.75 X X +autocfg@1.1.0 X X +backon@0.4.1 X +backtrace@0.3.69 X X +base64@0.21.5 X X +base64ct@1.6.0 X X +bimap@0.6.3 X X +bitflags@1.3.2 X X +bitflags@2.4.1 X X +bitvec@1.0.1 X +block-buffer@0.10.4 X X +bumpalo@3.14.0 X X +byteorder@1.5.0 X X +bytes@1.5.0 X +cc@1.0.83 X X +cfg-if@1.0.0 X X +chrono@0.4.31 X X +const-oid@0.9.6 X X +const-random@0.1.17 X X +const-random-macro@0.1.16 X X +core-foundation@0.9.4 X X +core-foundation-sys@0.8.6 X X +core2@0.4.0 X X +cpufeatures@0.2.11 X X +crc32fast@1.3.2 X X +crunchy@0.2.2 X +crypto-common@0.1.6 X X +darling@0.14.4 X +darling@0.20.3 X +darling_core@0.14.4 X +darling_core@0.20.3 X +darling_macro@0.14.4 X +darling_macro@0.20.3 X +dary_heap@0.3.6 X X +der@0.7.8 X X +deranged@0.3.10 X X +derive_builder@0.13.0 X X +derive_builder_core@0.13.0 X X +derive_builder_macro@0.13.0 X X +digest@0.10.7 X X +dlv-list@0.5.2 X X +either@1.9.0 X X +encoding_rs@0.8.33 X X X +equivalent@1.0.1 X X +fastrand@1.9.0 X X +fastrand@2.0.1 X X +flagset@0.4.4 X +fnv@1.0.7 X X +foreign-types@0.3.2 X X +foreign-types-shared@0.1.1 X X +form_urlencoded@1.2.1 X X +funty@2.0.0 X +futures@0.3.30 X X +futures-channel@0.3.30 X X +futures-core@0.3.30 X X +futures-executor@0.3.30 X X +futures-io@0.3.30 X X +futures-macro@0.3.30 X X +futures-sink@0.3.30 X X +futures-task@0.3.30 X X +futures-util@0.3.30 X X +generic-array@0.14.7 X +getrandom@0.2.11 X X +gimli@0.28.1 X X +h2@0.3.22 X +half@2.3.1 X X +hashbrown@0.13.2 X X +hashbrown@0.14.3 X X +heck@0.4.1 X X +hex@0.4.3 X X +hmac@0.12.1 X X +home@0.5.9 X X +http@0.2.11 X X +http-body@0.4.6 X +httparse@1.8.0 X X +httpdate@1.0.3 X X +hyper@0.14.28 X +hyper-rustls@0.24.2 X X X +hyper-tls@0.5.0 X X +iana-time-zone@0.1.58 X X +iana-time-zone-haiku@0.1.2 X X +iceberg@0.2.0 X +ident_case@1.0.1 X X +idna@0.5.0 X X +indexmap@2.1.0 X X +instant@0.1.12 X +ipnet@2.9.0 X X +itertools@0.12.0 X X +itoa@1.0.10 X X +js-sys@0.3.66 X X +jsonwebtoken@9.2.0 X +lazy_static@1.4.0 X X +libc@0.2.151 X X +libflate@2.0.0 X +libflate_lz77@2.0.0 X +libm@0.2.8 X X +linux-raw-sys@0.4.12 X X X +lock_api@0.4.11 X X +log@0.4.20 X X +md-5@0.10.6 X X +memchr@2.6.4 X X +mime@0.3.17 X X +miniz_oxide@0.7.1 X X X +mio@0.8.10 X +murmur3@0.5.2 X X +native-tls@0.2.11 X X +num@0.4.1 X X +num-bigint@0.4.4 X X +num-bigint-dig@0.8.4 X X +num-complex@0.4.4 X X +num-integer@0.1.45 X X +num-iter@0.1.43 X X +num-rational@0.4.1 X X +num-traits@0.2.17 X X +object@0.32.2 X X +once_cell@1.19.0 X X +opendal@0.44.0 X +openssl@0.10.62 X +openssl-macros@0.1.1 X X +openssl-probe@0.1.5 X X +openssl-sys@0.9.98 X +ordered-float@4.2.0 X +ordered-multimap@0.7.1 X +parking_lot@0.12.1 X X +parking_lot_core@0.9.9 X X +pem@3.0.3 X +pem-rfc7468@0.7.0 X X +percent-encoding@2.3.1 X X +pin-project@1.1.3 X X +pin-project-internal@1.1.3 X X +pin-project-lite@0.2.13 X X +pin-utils@0.1.0 X X +pkcs1@0.7.5 X X +pkcs8@0.10.2 X X +pkg-config@0.3.28 X X +powerfmt@0.2.0 X X +ppv-lite86@0.2.17 X X +proc-macro2@1.0.71 X X +quad-rand@0.2.1 X +quick-xml@0.30.0 X +quick-xml@0.31.0 X +quote@1.0.33 X X +radium@0.7.0 X +rand@0.8.5 X X +rand_chacha@0.3.1 X X +rand_core@0.6.4 X X +redox_syscall@0.4.1 X +regex-lite@0.1.5 X X +reqsign@0.14.6 X +reqwest@0.11.23 X X +ring@0.17.7 X +rle-decode-fast@1.0.3 X X +rsa@0.9.6 X X +rust-ini@0.20.0 X +rust_decimal@1.33.1 X +rustc-demangle@0.1.23 X X +rustix@0.38.28 X X X +rustls@0.21.10 X X X +rustls-native-certs@0.6.3 X X X +rustls-pemfile@1.0.4 X X X +rustls-webpki@0.101.7 X +rustversion@1.0.14 X X +ryu@1.0.16 X X +schannel@0.1.22 X +scopeguard@1.2.0 X X +sct@0.7.1 X X X +security-framework@2.9.2 X X +security-framework-sys@2.9.1 X X +serde@1.0.193 X X +serde_bytes@0.11.13 X X +serde_derive@1.0.193 X X +serde_json@1.0.108 X X +serde_repr@0.1.17 X X +serde_urlencoded@0.7.1 X X +serde_with@3.4.0 X X +serde_with_macros@3.4.0 X X +sha1@0.10.6 X X +sha2@0.10.8 X X +signature@2.2.0 X X +simple_asn1@0.6.2 X +slab@0.4.9 X +smallvec@1.11.2 X X +socket2@0.5.5 X X +spin@0.5.2 X +spin@0.9.8 X +spki@0.7.3 X X +strsim@0.10.0 X +strum@0.25.0 X +strum_macros@0.25.3 X +subtle@2.5.0 X +syn@1.0.109 X X +syn@2.0.43 X X +system-configuration@0.5.1 X X +system-configuration-sys@0.5.0 X X +tap@1.0.1 X +tempfile@3.8.1 X X +thiserror@1.0.52 X X +thiserror-impl@1.0.52 X X +time@0.3.31 X X +time-core@0.1.2 X X +time-macros@0.2.16 X X +tiny-keccak@2.0.2 X +tinyvec@1.6.0 X X X +tinyvec_macros@0.1.1 X X X +tokio@1.35.1 X +tokio-macros@2.2.0 X +tokio-native-tls@0.3.1 X +tokio-rustls@0.24.1 X X +tokio-util@0.7.10 X +tower-service@0.3.2 X +tracing@0.1.40 X +tracing-core@0.1.32 X +try-lock@0.2.5 X +typed-builder@0.16.2 X X +typed-builder@0.18.0 X X +typed-builder-macro@0.16.2 X X +typed-builder-macro@0.18.0 X X +typenum@1.17.0 X X +unicode-bidi@0.3.14 X X +unicode-ident@1.0.12 X X X +unicode-normalization@0.1.22 X X +untrusted@0.9.0 X +url@2.5.0 X X +urlencoding@2.1.3 X +uuid@1.6.1 X X +vcpkg@0.2.15 X X +version_check@0.9.4 X X +want@0.3.1 X +wasi@0.11.0+wasi-snapshot-preview1 X X X +wasm-bindgen@0.2.89 X X +wasm-bindgen-backend@0.2.89 X X +wasm-bindgen-futures@0.4.39 X X +wasm-bindgen-macro@0.2.89 X X +wasm-bindgen-macro-support@0.2.89 X X +wasm-bindgen-shared@0.2.89 X X +wasm-streams@0.3.0 X X +web-sys@0.3.66 X X +windows-core@0.51.1 X X +windows-sys@0.48.0 X X +windows-sys@0.52.0 X X +windows-targets@0.48.5 X X +windows-targets@0.52.0 X X +windows_aarch64_gnullvm@0.48.5 X X +windows_aarch64_gnullvm@0.52.0 X X +windows_aarch64_msvc@0.48.5 X X +windows_aarch64_msvc@0.52.0 X X +windows_i686_gnu@0.48.5 X X +windows_i686_gnu@0.52.0 X X +windows_i686_msvc@0.48.5 X X +windows_i686_msvc@0.52.0 X X +windows_x86_64_gnu@0.48.5 X X +windows_x86_64_gnu@0.52.0 X X +windows_x86_64_gnullvm@0.48.5 X X +windows_x86_64_gnullvm@0.52.0 X X +windows_x86_64_msvc@0.48.5 X X +windows_x86_64_msvc@0.52.0 X X +winreg@0.50.0 X +wyz@0.5.1 X +zerocopy@0.7.32 X X X +zeroize@1.7.0 X X diff --git a/crates/iceberg/src/avro/mod.rs b/crates/iceberg/src/avro/mod.rs new file mode 100644 index 0000000..bdccb2f --- /dev/null +++ b/crates/iceberg/src/avro/mod.rs @@ -0,0 +1,21 @@ +// 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. + +//! Avro related codes. +#[allow(dead_code)] +mod schema; +pub(crate) use schema::*; diff --git a/crates/iceberg/src/avro/schema.rs b/crates/iceberg/src/avro/schema.rs new file mode 100644 index 0000000..636f128 --- /dev/null +++ b/crates/iceberg/src/avro/schema.rs @@ -0,0 +1,955 @@ +// 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. + +//! Conversion between iceberg and avro schema. +use std::collections::BTreeMap; + +use crate::spec::{ + visit_schema, ListType, MapType, NestedField, NestedFieldRef, PrimitiveType, Schema, + SchemaVisitor, StructType, Type, +}; +use crate::{ensure_data_valid, Error, ErrorKind, Result}; +use apache_avro::schema::{ + DecimalSchema, FixedSchema, Name, RecordField as AvroRecordField, RecordFieldOrder, + RecordSchema, UnionSchema, +}; +use apache_avro::Schema as AvroSchema; +use itertools::{Either, Itertools}; +use serde_json::{Number, Value}; + +const FILED_ID_PROP: &str = "field-id"; +const UUID_BYTES: usize = 16; +const UUID_LOGICAL_TYPE: &str = "uuid"; +// # TODO: https://github.com/apache/iceberg-rust/issues/86 +// This const may better to maintain in avro-rs. +const LOGICAL_TYPE: &str = "logicalType"; + +struct SchemaToAvroSchema { + schema: String, +} + +type AvroSchemaOrField = Either; + +impl SchemaVisitor for SchemaToAvroSchema { + type T = AvroSchemaOrField; + + fn schema(&mut self, _schema: &Schema, value: AvroSchemaOrField) -> Result { + let mut avro_schema = value.unwrap_left(); + + if let AvroSchema::Record(record) = &mut avro_schema { + record.name = Name::from(self.schema.as_str()); + } else { + return Err(Error::new( + ErrorKind::Unexpected, + "Schema result must be avro record!", + )); + } + + Ok(Either::Left(avro_schema)) + } + + fn field( + &mut self, + field: &NestedFieldRef, + avro_schema: AvroSchemaOrField, + ) -> Result { + let mut field_schema = avro_schema.unwrap_left(); + if let AvroSchema::Record(record) = &mut field_schema { + record.name = Name::from(format!("r{}", field.id).as_str()); + } + + if !field.required { + field_schema = avro_optional(field_schema)?; + } + + let mut avro_record_field = AvroRecordField { + name: field.name.clone(), + schema: field_schema, + order: RecordFieldOrder::Ignore, + position: 0, + doc: field.doc.clone(), + aliases: None, + default: None, + custom_attributes: Default::default(), + }; + + if !field.required { + avro_record_field.default = Some(Value::Null); + } + avro_record_field.custom_attributes.insert( + FILED_ID_PROP.to_string(), + Value::Number(Number::from(field.id)), + ); + + Ok(Either::Right(avro_record_field)) + } + + fn r#struct( + &mut self, + _struct: &StructType, + results: Vec, + ) -> Result { + let avro_fields = results.into_iter().map(|r| r.unwrap_right()).collect_vec(); + + Ok(Either::Left( + // The name of this record schema should be determined later, by schema name or field + // name, here we use a temporary placeholder to do it. + avro_record_schema("null", avro_fields)?, + )) + } + + fn list(&mut self, list: &ListType, value: AvroSchemaOrField) -> Result { + let mut field_schema = value.unwrap_left(); + + if let AvroSchema::Record(record) = &mut field_schema { + record.name = Name::from(format!("r{}", list.element_field.id).as_str()); + } + + if !list.element_field.required { + field_schema = avro_optional(field_schema)?; + } + + // TODO: We need to add element id prop here, but rust's avro schema doesn't support property except record schema. + Ok(Either::Left(AvroSchema::Array(Box::new(field_schema)))) + } + + fn map( + &mut self, + map: &MapType, + key_value: AvroSchemaOrField, + value: AvroSchemaOrField, + ) -> Result { + let key_field_schema = key_value.unwrap_left(); + let mut value_field_schema = value.unwrap_left(); + if !map.value_field.required { + value_field_schema = avro_optional(value_field_schema)?; + } + + if matches!(key_field_schema, AvroSchema::String) { + Ok(Either::Left(AvroSchema::Map(Box::new(value_field_schema)))) + } else { + // Avro map requires that key must be string type. Here we convert it to array if key is + // not string type. + let key_field = { + let mut field = AvroRecordField { + name: map.key_field.name.clone(), + doc: None, + aliases: None, + default: None, + schema: key_field_schema, + order: RecordFieldOrder::Ascending, + position: 0, + custom_attributes: Default::default(), + }; + field.custom_attributes.insert( + FILED_ID_PROP.to_string(), + Value::Number(Number::from(map.key_field.id)), + ); + field + }; + + let value_field = { + let mut field = AvroRecordField { + name: map.value_field.name.clone(), + doc: None, + aliases: None, + default: None, + schema: value_field_schema, + order: RecordFieldOrder::Ignore, + position: 0, + custom_attributes: Default::default(), + }; + field.custom_attributes.insert( + FILED_ID_PROP.to_string(), + Value::Number(Number::from(map.value_field.id)), + ); + field + }; + + let fields = vec![key_field, value_field]; + let item_avro_schema = avro_record_schema( + format!("k{}_v{}", map.key_field.id, map.value_field.id).as_str(), + fields, + )?; + + Ok(Either::Left(AvroSchema::Array(item_avro_schema.into()))) + } + } + + fn primitive(&mut self, p: &PrimitiveType) -> Result { + let avro_schema = match p { + PrimitiveType::Boolean => AvroSchema::Boolean, + PrimitiveType::Int => AvroSchema::Int, + PrimitiveType::Long => AvroSchema::Long, + PrimitiveType::Float => AvroSchema::Float, + PrimitiveType::Double => AvroSchema::Double, + PrimitiveType::Date => AvroSchema::Date, + PrimitiveType::Time => AvroSchema::TimeMicros, + PrimitiveType::Timestamp => AvroSchema::TimestampMicros, + PrimitiveType::Timestamptz => AvroSchema::TimestampMicros, + PrimitiveType::String => AvroSchema::String, + PrimitiveType::Uuid => avro_fixed_schema(UUID_BYTES, Some(UUID_LOGICAL_TYPE))?, + PrimitiveType::Fixed(len) => avro_fixed_schema((*len) as usize, None)?, + PrimitiveType::Binary => AvroSchema::Bytes, + PrimitiveType::Decimal { precision, scale } => { + avro_decimal_schema(*precision as usize, *scale as usize)? + } + }; + Ok(Either::Left(avro_schema)) + } +} + +/// Converting iceberg schema to avro schema. +pub(crate) fn schema_to_avro_schema(name: impl ToString, schema: &Schema) -> Result { + let mut converter = SchemaToAvroSchema { + schema: name.to_string(), + }; + + visit_schema(schema, &mut converter).map(Either::unwrap_left) +} + +fn avro_record_schema(name: &str, fields: Vec) -> Result { + let lookup = fields + .iter() + .enumerate() + .map(|f| (f.1.name.clone(), f.0)) + .collect(); + + Ok(AvroSchema::Record(RecordSchema { + name: Name::new(name)?, + aliases: None, + doc: None, + fields, + lookup, + attributes: Default::default(), + })) +} + +pub(crate) fn avro_fixed_schema(len: usize, logical_type: Option<&str>) -> Result { + let attributes = if let Some(logical_type) = logical_type { + BTreeMap::from([( + LOGICAL_TYPE.to_string(), + Value::String(logical_type.to_string()), + )]) + } else { + Default::default() + }; + Ok(AvroSchema::Fixed(FixedSchema { + name: Name::new(format!("fixed_{len}").as_str())?, + aliases: None, + doc: None, + size: len, + attributes, + })) +} + +pub(crate) fn avro_decimal_schema(precision: usize, scale: usize) -> Result { + Ok(AvroSchema::Decimal(DecimalSchema { + precision, + scale, + inner: Box::new(AvroSchema::Bytes), + })) +} + +fn avro_optional(avro_schema: AvroSchema) -> Result { + Ok(AvroSchema::Union(UnionSchema::new(vec![ + AvroSchema::Null, + avro_schema, + ])?)) +} + +fn is_avro_optional(avro_schema: &AvroSchema) -> bool { + match avro_schema { + AvroSchema::Union(union) => union.is_nullable(), + _ => false, + } +} + +/// Post order avro schema visitor. +pub(crate) trait AvroSchemaVisitor { + type T; + + fn record(&mut self, record: &RecordSchema, fields: Vec) -> Result; + + fn union(&mut self, union: &UnionSchema, options: Vec) -> Result; + + fn array(&mut self, array: &AvroSchema, item: Self::T) -> Result; + fn map(&mut self, map: &AvroSchema, value: Self::T) -> Result; + + fn primitive(&mut self, schema: &AvroSchema) -> Result; +} + +/// Visit avro schema in post order visitor. +pub(crate) fn visit(schema: &AvroSchema, visitor: &mut V) -> Result { + match schema { + AvroSchema::Record(record) => { + let field_results = record + .fields + .iter() + .map(|f| visit(&f.schema, visitor)) + .collect::>>()?; + + visitor.record(record, field_results) + } + AvroSchema::Union(union) => { + let option_results = union + .variants() + .iter() + .map(|f| visit(f, visitor)) + .collect::>>()?; + + visitor.union(union, option_results) + } + AvroSchema::Array(item) => { + let item_result = visit(item, visitor)?; + visitor.array(schema, item_result) + } + AvroSchema::Map(inner) => { + let item_result = visit(inner, visitor)?; + visitor.map(schema, item_result) + } + schema => visitor.primitive(schema), + } +} + +struct AvroSchemaToSchema { + next_id: i32, +} + +impl AvroSchemaToSchema { + fn next_field_id(&mut self) -> i32 { + self.next_id += 1; + self.next_id + } +} + +impl AvroSchemaVisitor for AvroSchemaToSchema { + // Only `AvroSchema::Null` will return `None` + type T = Option; + + fn record( + &mut self, + record: &RecordSchema, + field_types: Vec>, + ) -> Result> { + let mut fields = Vec::with_capacity(field_types.len()); + for (avro_field, typ) in record.fields.iter().zip_eq(field_types) { + let field_id = avro_field + .custom_attributes + .get(FILED_ID_PROP) + .and_then(Value::as_i64) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't convert field, missing field id: {avro_field:?}"), + ) + })?; + + let optional = is_avro_optional(&avro_field.schema); + + let mut field = if optional { + NestedField::optional(field_id as i32, &avro_field.name, typ.unwrap()) + } else { + NestedField::required(field_id as i32, &avro_field.name, typ.unwrap()) + }; + + if let Some(doc) = &avro_field.doc { + field = field.with_doc(doc); + } + + fields.push(field.into()); + } + + Ok(Some(Type::Struct(StructType::new(fields)))) + } + + fn union( + &mut self, + union: &UnionSchema, + mut options: Vec>, + ) -> Result> { + ensure_data_valid!( + options.len() <= 2 && !options.is_empty(), + "Can't convert avro union type {:?} to iceberg.", + union + ); + + if options.len() > 1 { + ensure_data_valid!( + options[0].is_none(), + "Can't convert avro union type {:?} to iceberg.", + union + ); + } + + if options.len() == 1 { + Ok(Some(options.remove(0).unwrap())) + } else { + Ok(Some(options.remove(1).unwrap())) + } + } + + fn array(&mut self, array: &AvroSchema, item: Option) -> Result { + if let AvroSchema::Array(item_schema) = array { + let element_field = NestedField::list_element( + self.next_field_id(), + item.unwrap(), + !is_avro_optional(item_schema), + ) + .into(); + Ok(Some(Type::List(ListType { element_field }))) + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Expected avro array schema, but {array}", + )) + } + } + + fn map(&mut self, map: &AvroSchema, value: Option) -> Result> { + if let AvroSchema::Map(value_schema) = map { + // Due to avro rust implementation's limitation, we can't store attributes in map schema, + // we will fix it later when it has been resolved. + let key_field = NestedField::map_key_element( + self.next_field_id(), + Type::Primitive(PrimitiveType::String), + ); + let value_field = NestedField::map_value_element( + self.next_field_id(), + value.unwrap(), + !is_avro_optional(value_schema), + ); + Ok(Some(Type::Map(MapType { + key_field: key_field.into(), + value_field: value_field.into(), + }))) + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Expected avro map schema, but {map}", + )) + } + } + + fn primitive(&mut self, schema: &AvroSchema) -> Result> { + let typ = match schema { + AvroSchema::Decimal(decimal) => { + Type::decimal(decimal.precision as u32, decimal.scale as u32)? + } + AvroSchema::Date => Type::Primitive(PrimitiveType::Date), + AvroSchema::TimeMicros => Type::Primitive(PrimitiveType::Time), + AvroSchema::TimestampMicros => Type::Primitive(PrimitiveType::Timestamp), + AvroSchema::Boolean => Type::Primitive(PrimitiveType::Boolean), + AvroSchema::Int => Type::Primitive(PrimitiveType::Int), + AvroSchema::Long => Type::Primitive(PrimitiveType::Long), + AvroSchema::Float => Type::Primitive(PrimitiveType::Float), + AvroSchema::Double => Type::Primitive(PrimitiveType::Double), + AvroSchema::String | AvroSchema::Enum(_) => Type::Primitive(PrimitiveType::String), + AvroSchema::Fixed(fixed) => { + if let Some(logical_type) = fixed.attributes.get(LOGICAL_TYPE) { + let logical_type = logical_type.as_str().ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "logicalType in attributes of avro schema is not a string type", + ) + })?; + match logical_type { + UUID_LOGICAL_TYPE => Type::Primitive(PrimitiveType::Uuid), + ty => { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Logical type {ty} is not support in iceberg primitive type.", + ), + )) + } + } + } else { + Type::Primitive(PrimitiveType::Fixed(fixed.size as u64)) + } + } + AvroSchema::Bytes => Type::Primitive(PrimitiveType::Binary), + AvroSchema::Null => return Ok(None), + _ => { + return Err(Error::new( + ErrorKind::Unexpected, + "Unable to convert avro {schema} to iceberg primitive type.", + )) + } + }; + + Ok(Some(typ)) + } +} + +/// Converts avro schema to iceberg schema. +pub(crate) fn avro_schema_to_schema(avro_schema: &AvroSchema) -> Result { + if let AvroSchema::Record(_) = avro_schema { + let mut converter = AvroSchemaToSchema { next_id: 0 }; + let typ = visit(avro_schema, &mut converter)?.expect("Iceberg schema should not be none."); + if let Type::Struct(s) = typ { + Schema::builder() + .with_fields(s.fields().iter().cloned()) + .build() + } else { + Err(Error::new( + ErrorKind::Unexpected, + format!("Expected to convert avro record schema to struct type, but {typ}"), + )) + } + } else { + Err(Error::new( + ErrorKind::DataInvalid, + "Can't convert non record avro schema to iceberg schema: {avro_schema}", + )) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::avro::schema::AvroSchemaToSchema; + use crate::spec::{ListType, MapType, NestedField, PrimitiveType, Schema, StructType, Type}; + use apache_avro::schema::{Namespace, UnionSchema}; + use apache_avro::Schema as AvroSchema; + use std::fs::read_to_string; + + fn read_test_data_file_to_avro_schema(filename: &str) -> AvroSchema { + let input = read_to_string(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + filename + )) + .unwrap(); + + AvroSchema::parse_str(input.as_str()).unwrap() + } + + fn check_schema_conversion( + avro_schema: AvroSchema, + expected_iceberg_schema: Schema, + check_avro_to_iceberg: bool, + ) { + if check_avro_to_iceberg { + let converted_iceberg_schema = avro_schema_to_schema(&avro_schema).unwrap(); + assert_eq!(expected_iceberg_schema, converted_iceberg_schema); + } + + let converted_avro_schema = schema_to_avro_schema( + avro_schema.name().unwrap().fullname(Namespace::None), + &expected_iceberg_schema, + ) + .unwrap(); + assert_eq!(avro_schema, converted_avro_schema); + } + + #[test] + fn test_manifest_file_v1_schema() { + let fields = vec![ + NestedField::required(500, "manifest_path", PrimitiveType::String.into()) + .with_doc("Location URI with FS scheme") + .into(), + NestedField::required(501, "manifest_length", PrimitiveType::Long.into()) + .with_doc("Total file size in bytes") + .into(), + NestedField::required(502, "partition_spec_id", PrimitiveType::Int.into()) + .with_doc("Spec ID used to write") + .into(), + NestedField::optional(503, "added_snapshot_id", PrimitiveType::Long.into()) + .with_doc("Snapshot ID that added the manifest") + .into(), + NestedField::optional(504, "added_data_files_count", PrimitiveType::Int.into()) + .with_doc("Added entry count") + .into(), + NestedField::optional(505, "existing_data_files_count", PrimitiveType::Int.into()) + .with_doc("Existing entry count") + .into(), + NestedField::optional(506, "deleted_data_files_count", PrimitiveType::Int.into()) + .with_doc("Deleted entry count") + .into(), + NestedField::optional( + 507, + "partitions", + ListType { + element_field: NestedField::list_element( + 508, + StructType::new(vec![ + NestedField::required( + 509, + "contains_null", + PrimitiveType::Boolean.into(), + ) + .with_doc("True if any file has a null partition value") + .into(), + NestedField::optional( + 518, + "contains_nan", + PrimitiveType::Boolean.into(), + ) + .with_doc("True if any file has a nan partition value") + .into(), + NestedField::optional(510, "lower_bound", PrimitiveType::Binary.into()) + .with_doc("Partition lower bound for all files") + .into(), + NestedField::optional(511, "upper_bound", PrimitiveType::Binary.into()) + .with_doc("Partition upper bound for all files") + .into(), + ]) + .into(), + true, + ) + .into(), + } + .into(), + ) + .with_doc("Summary for each partition") + .into(), + NestedField::optional(512, "added_rows_count", PrimitiveType::Long.into()) + .with_doc("Added rows count") + .into(), + NestedField::optional(513, "existing_rows_count", PrimitiveType::Long.into()) + .with_doc("Existing rows count") + .into(), + NestedField::optional(514, "deleted_rows_count", PrimitiveType::Long.into()) + .with_doc("Deleted rows count") + .into(), + ]; + + let iceberg_schema = Schema::builder().with_fields(fields).build().unwrap(); + check_schema_conversion( + read_test_data_file_to_avro_schema("avro_schema_manifest_file_v1.json"), + iceberg_schema, + false, + ); + } + + #[test] + fn test_avro_list_required_primitive() { + let avro_schema = { + AvroSchema::parse_str( + r#" +{ + "type": "record", + "name": "avro_schema", + "fields": [ + { + "name": "array_with_string", + "type": { + "type": "array", + "items": "string", + "default": [], + "element-id": 101 + }, + "field-id": 100 + } + ] +}"#, + ) + .unwrap() + }; + + let iceberg_schema = { + Schema::builder() + .with_fields(vec![NestedField::required( + 100, + "array_with_string", + ListType { + element_field: NestedField::list_element( + 101, + PrimitiveType::String.into(), + true, + ) + .into(), + } + .into(), + ) + .into()]) + .build() + .unwrap() + }; + + check_schema_conversion(avro_schema, iceberg_schema, false); + } + + #[test] + fn test_avro_list_wrapped_primitive() { + let avro_schema = { + AvroSchema::parse_str( + r#" +{ + "type": "record", + "name": "avro_schema", + "fields": [ + { + "name": "array_with_string", + "type": { + "type": "array", + "items": {"type": "string"}, + "default": [], + "element-id": 101 + }, + "field-id": 100 + } + ] +} +"#, + ) + .unwrap() + }; + + let iceberg_schema = { + Schema::builder() + .with_fields(vec![NestedField::required( + 100, + "array_with_string", + ListType { + element_field: NestedField::list_element( + 101, + PrimitiveType::String.into(), + true, + ) + .into(), + } + .into(), + ) + .into()]) + .build() + .unwrap() + }; + + check_schema_conversion(avro_schema, iceberg_schema, false); + } + + #[test] + fn test_avro_list_required_record() { + let avro_schema = { + AvroSchema::parse_str( + r#" +{ + "type": "record", + "name": "avro_schema", + "fields": [ + { + "name": "array_with_record", + "type": { + "type": "array", + "items": { + "type": "record", + "name": "r101", + "fields": [ + { + "name": "contains_null", + "type": "boolean", + "field-id": 102 + }, + { + "name": "contains_nan", + "type": ["null", "boolean"], + "field-id": 103 + } + ] + }, + "element-id": 101 + }, + "field-id": 100 + } + ] +} +"#, + ) + .unwrap() + }; + + let iceberg_schema = { + Schema::builder() + .with_fields(vec![NestedField::required( + 100, + "array_with_record", + ListType { + element_field: NestedField::list_element( + 101, + StructType::new(vec![ + NestedField::required( + 102, + "contains_null", + PrimitiveType::Boolean.into(), + ) + .into(), + NestedField::optional( + 103, + "contains_nan", + PrimitiveType::Boolean.into(), + ) + .into(), + ]) + .into(), + true, + ) + .into(), + } + .into(), + ) + .into()]) + .build() + .unwrap() + }; + + check_schema_conversion(avro_schema, iceberg_schema, false); + } + + #[test] + fn test_resolve_union() { + let avro_schema = UnionSchema::new(vec![ + AvroSchema::Null, + AvroSchema::String, + AvroSchema::Boolean, + ]) + .unwrap(); + + let mut converter = AvroSchemaToSchema { next_id: 0 }; + + let options = avro_schema + .variants() + .iter() + .map(|v| converter.primitive(v).unwrap()) + .collect(); + assert!(converter.union(&avro_schema, options).is_err()); + } + + #[test] + fn test_string_type() { + let mut converter = AvroSchemaToSchema { next_id: 0 }; + let avro_schema = AvroSchema::String; + + assert_eq!( + Some(PrimitiveType::String.into()), + converter.primitive(&avro_schema).unwrap() + ); + } + + #[test] + fn test_map_type() { + let avro_schema = { + AvroSchema::parse_str( + r#" +{ + "type": "map", + "values": ["null", "long"], + "key-id": 101, + "value-id": 102 +} +"#, + ) + .unwrap() + }; + + let mut converter = AvroSchemaToSchema { next_id: 0 }; + let iceberg_type = Type::Map(MapType { + key_field: NestedField::map_key_element(1, PrimitiveType::String.into()).into(), + value_field: NestedField::map_value_element(2, PrimitiveType::Long.into(), false) + .into(), + }); + + assert_eq!( + iceberg_type, + converter + .map(&avro_schema, Some(PrimitiveType::Long.into())) + .unwrap() + .unwrap() + ); + } + + #[test] + fn test_fixed_type() { + let avro_schema = { + AvroSchema::parse_str( + r#" + {"name": "test", "type": "fixed", "size": 22} + "#, + ) + .unwrap() + }; + + let mut converter = AvroSchemaToSchema { next_id: 0 }; + + let iceberg_type = Type::from(PrimitiveType::Fixed(22)); + + assert_eq!( + iceberg_type, + converter.primitive(&avro_schema).unwrap().unwrap() + ); + } + + #[test] + fn test_unknown_primitive() { + let mut converter = AvroSchemaToSchema { next_id: 0 }; + + assert!(converter.primitive(&AvroSchema::Duration).is_err()); + } + + #[test] + fn test_no_field_id() { + let avro_schema = { + AvroSchema::parse_str( + r#" +{ + "type": "record", + "name": "avro_schema", + "fields": [ + { + "name": "array_with_string", + "type": "string" + } + ] +} +"#, + ) + .unwrap() + }; + + assert!(avro_schema_to_schema(&avro_schema).is_err()); + } + + #[test] + fn test_decimal_type() { + let avro_schema = { + AvroSchema::parse_str( + r#" + {"type": "bytes", "logicalType": "decimal", "precision": 25, "scale": 19} + "#, + ) + .unwrap() + }; + + let mut converter = AvroSchemaToSchema { next_id: 0 }; + + assert_eq!( + Type::decimal(25, 19).unwrap(), + converter.primitive(&avro_schema).unwrap().unwrap() + ); + } + + #[test] + fn test_date_type() { + let mut converter = AvroSchemaToSchema { next_id: 0 }; + + assert_eq!( + Type::from(PrimitiveType::Date), + converter.primitive(&AvroSchema::Date).unwrap().unwrap() + ); + } +} diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs new file mode 100644 index 0000000..708e6bf --- /dev/null +++ b/crates/iceberg/src/catalog/mod.rs @@ -0,0 +1,1068 @@ +// 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. + +//! Catalog API for Apache Iceberg + +use crate::spec::{ + FormatVersion, Schema, Snapshot, SnapshotReference, SortOrder, UnboundPartitionSpec, +}; +use crate::table::Table; +use crate::{Error, ErrorKind, Result}; +use async_trait::async_trait; +use serde_derive::{Deserialize, Serialize}; +use std::collections::HashMap; +use std::fmt::Debug; +use std::mem::take; +use std::ops::Deref; +use typed_builder::TypedBuilder; +use urlencoding::encode; +use uuid::Uuid; + +/// The catalog API for Iceberg Rust. +#[async_trait] +pub trait Catalog: Debug + Sync + Send { + /// List namespaces from table. + async fn list_namespaces(&self, parent: Option<&NamespaceIdent>) + -> Result>; + + /// Create a new namespace inside the catalog. + async fn create_namespace( + &self, + namespace: &NamespaceIdent, + properties: HashMap, + ) -> Result; + + /// Get a namespace information from the catalog. + async fn get_namespace(&self, namespace: &NamespaceIdent) -> Result; + + /// Check if namespace exists in catalog. + async fn namespace_exists(&self, namespace: &NamespaceIdent) -> Result; + + /// Update a namespace inside the catalog. + /// + /// # Behavior + /// + /// The properties must be the full set of namespace. + async fn update_namespace( + &self, + namespace: &NamespaceIdent, + properties: HashMap, + ) -> Result<()>; + + /// Drop a namespace from the catalog. + async fn drop_namespace(&self, namespace: &NamespaceIdent) -> Result<()>; + + /// List tables from namespace. + async fn list_tables(&self, namespace: &NamespaceIdent) -> Result>; + + /// Create a new table inside the namespace. + async fn create_table( + &self, + namespace: &NamespaceIdent, + creation: TableCreation, + ) -> Result; + + /// Load table from the catalog. + async fn load_table(&self, table: &TableIdent) -> Result
; + + /// Drop a table from the catalog. + async fn drop_table(&self, table: &TableIdent) -> Result<()>; + + /// Check if a table exists in the catalog. + async fn stat_table(&self, table: &TableIdent) -> Result; + + /// Rename a table in the catalog. + async fn rename_table(&self, src: &TableIdent, dest: &TableIdent) -> Result<()>; + + /// Update a table to the catalog. + async fn update_table(&self, commit: TableCommit) -> Result
; +} + +/// NamespaceIdent represents the identifier of a namespace in the catalog. +/// +/// The namespace identifier is a list of strings, where each string is a +/// component of the namespace. It's catalog implementer's responsibility to +/// handle the namespace identifier correctly. +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct NamespaceIdent(Vec); + +impl NamespaceIdent { + /// Create a new namespace identifier with only one level. + pub fn new(name: String) -> Self { + Self(vec![name]) + } + + /// Create a multi-level namespace identifier from vector. + pub fn from_vec(names: Vec) -> Result { + if names.is_empty() { + return Err(Error::new( + ErrorKind::DataInvalid, + "Namespace identifier can't be empty!", + )); + } + Ok(Self(names)) + } + + /// Try to create namespace identifier from an iterator of string. + pub fn from_strs(iter: impl IntoIterator) -> Result { + Self::from_vec(iter.into_iter().map(|s| s.to_string()).collect()) + } + + /// Returns url encoded format. + pub fn encode_in_url(&self) -> String { + encode(&self.as_ref().join("\u{1F}")).to_string() + } + + /// Returns inner strings. + pub fn inner(self) -> Vec { + self.0 + } +} + +impl AsRef> for NamespaceIdent { + fn as_ref(&self) -> &Vec { + &self.0 + } +} + +impl Deref for NamespaceIdent { + type Target = [String]; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +/// Namespace represents a namespace in the catalog. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct Namespace { + name: NamespaceIdent, + properties: HashMap, +} + +impl Namespace { + /// Create a new namespace. + pub fn new(name: NamespaceIdent) -> Self { + Self::with_properties(name, HashMap::default()) + } + + /// Create a new namespace with properties. + pub fn with_properties(name: NamespaceIdent, properties: HashMap) -> Self { + Self { name, properties } + } + + /// Get the name of the namespace. + pub fn name(&self) -> &NamespaceIdent { + &self.name + } + + /// Get the properties of the namespace. + pub fn properties(&self) -> &HashMap { + &self.properties + } +} + +/// TableIdent represents the identifier of a table in the catalog. +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, Hash)] +pub struct TableIdent { + /// Namespace of the table. + pub namespace: NamespaceIdent, + /// Table name. + pub name: String, +} + +impl TableIdent { + /// Create a new table identifier. + pub fn new(namespace: NamespaceIdent, name: String) -> Self { + Self { namespace, name } + } + + /// Get the namespace of the table. + pub fn namespace(&self) -> &NamespaceIdent { + &self.namespace + } + + /// Get the name of the table. + pub fn name(&self) -> &str { + &self.name + } + + /// Try to create table identifier from an iterator of string. + pub fn from_strs(iter: impl IntoIterator) -> Result { + let mut vec: Vec = iter.into_iter().map(|s| s.to_string()).collect(); + let table_name = vec.pop().ok_or_else(|| { + Error::new(ErrorKind::DataInvalid, "Table identifier can't be empty!") + })?; + let namespace_ident = NamespaceIdent::from_vec(vec)?; + + Ok(Self { + namespace: namespace_ident, + name: table_name, + }) + } +} + +/// TableCreation represents the creation of a table in the catalog. +#[derive(Debug, TypedBuilder)] +pub struct TableCreation { + /// The name of the table. + pub name: String, + /// The location of the table. + #[builder(default, setter(strip_option))] + pub location: Option, + /// The schema of the table. + pub schema: Schema, + /// The partition spec of the table, could be None. + #[builder(default, setter(strip_option))] + pub partition_spec: Option, + /// The sort order of the table. + #[builder(default, setter(strip_option))] + pub sort_order: Option, + /// The properties of the table. + #[builder(default)] + pub properties: HashMap, +} + +/// TableCommit represents the commit of a table in the catalog. +#[derive(Debug, TypedBuilder)] +#[builder(build_method(vis = "pub(crate)"))] +pub struct TableCommit { + /// The table ident. + ident: TableIdent, + /// The requirements of the table. + /// + /// Commit will fail if the requirements are not met. + requirements: Vec, + /// The updates of the table. + updates: Vec, +} + +impl TableCommit { + /// Return the table identifier. + pub fn identifier(&self) -> &TableIdent { + &self.ident + } + + /// Take all requirements. + pub fn take_requirements(&mut self) -> Vec { + take(&mut self.requirements) + } + + /// Take all updates. + pub fn take_updates(&mut self) -> Vec { + take(&mut self.updates) + } +} + +/// TableRequirement represents a requirement for a table in the catalog. +#[derive(Debug, Serialize, Deserialize, PartialEq)] +#[serde(tag = "type")] +pub enum TableRequirement { + /// The table must not already exist; used for create transactions + #[serde(rename = "assert-create")] + NotExist, + /// The table UUID must match the requirement. + #[serde(rename = "assert-table-uuid")] + UuidMatch { + /// Uuid of original table. + uuid: Uuid, + }, + /// The table branch or tag identified by the requirement's `reference` must + /// reference the requirement's `snapshot-id`. + #[serde(rename = "assert-ref-snapshot-id")] + RefSnapshotIdMatch { + /// The reference of the table to assert. + r#ref: String, + /// The snapshot id of the table to assert. + /// If the id is `None`, the ref must not already exist. + #[serde(rename = "snapshot-id")] + snapshot_id: Option, + }, + /// The table's last assigned column id must match the requirement. + #[serde(rename = "assert-last-assigned-field-id")] + LastAssignedFieldIdMatch { + /// The last assigned field id of the table to assert. + #[serde(rename = "last-assigned-field-id")] + last_assigned_field_id: i64, + }, + /// The table's current schema id must match the requirement. + #[serde(rename = "assert-current-schema-id")] + CurrentSchemaIdMatch { + /// Current schema id of the table to assert. + #[serde(rename = "current-schema-id")] + current_schema_id: i64, + }, + /// The table's last assigned partition id must match the + /// requirement. + #[serde(rename = "assert-last-assigned-partition-id")] + LastAssignedPartitionIdMatch { + /// Last assigned partition id of the table to assert. + #[serde(rename = "last-assigned-partition-id")] + last_assigned_partition_id: i64, + }, + /// The table's default spec id must match the requirement. + #[serde(rename = "assert-default-spec-id")] + DefaultSpecIdMatch { + /// Default spec id of the table to assert. + #[serde(rename = "default-spec-id")] + default_spec_id: i64, + }, + /// The table's default sort order id must match the requirement. + #[serde(rename = "assert-default-sort-order-id")] + DefaultSortOrderIdMatch { + /// Default sort order id of the table to assert. + #[serde(rename = "default-sort-order-id")] + default_sort_order_id: i64, + }, +} + +/// TableUpdate represents an update to a table in the catalog. +#[derive(Debug, Serialize, Deserialize, PartialEq)] +#[serde(tag = "action", rename_all = "kebab-case")] +pub enum TableUpdate { + /// Upgrade table's format version + #[serde(rename_all = "kebab-case")] + UpgradeFormatVersion { + /// Target format upgrade to. + format_version: FormatVersion, + }, + /// Assign a new UUID to the table + #[serde(rename_all = "kebab-case")] + AssignUuid { + /// The new UUID to assign. + uuid: Uuid, + }, + /// Add a new schema to the table + #[serde(rename_all = "kebab-case")] + AddSchema { + /// The schema to add. + schema: Schema, + /// The last column id of the table. + last_column_id: Option, + }, + /// Set table's current schema + #[serde(rename_all = "kebab-case")] + SetCurrentSchema { + /// Schema ID to set as current, or -1 to set last added schema + schema_id: i32, + }, + /// Add a new partition spec to the table + AddSpec { + /// The partition spec to add. + spec: UnboundPartitionSpec, + }, + /// Set table's default spec + #[serde(rename_all = "kebab-case")] + SetDefaultSpec { + /// Partition spec ID to set as the default, or -1 to set last added spec + spec_id: i32, + }, + /// Add sort order to table. + #[serde(rename_all = "kebab-case")] + AddSortOrder { + /// Sort order to add. + sort_order: SortOrder, + }, + /// Set table's default sort order + #[serde(rename_all = "kebab-case")] + SetDefaultSortOrder { + /// Sort order ID to set as the default, or -1 to set last added sort order + sort_order_id: i32, + }, + /// Add snapshot to table. + #[serde(rename_all = "kebab-case")] + AddSnapshot { + /// Snapshot to add. + snapshot: Snapshot, + }, + /// Set table's snapshot ref. + #[serde(rename_all = "kebab-case")] + SetSnapshotRef { + /// Name of snapshot reference to set. + ref_name: String, + /// Snapshot reference to set. + #[serde(flatten)] + reference: SnapshotReference, + }, + /// Remove table's snapshots + #[serde(rename_all = "kebab-case")] + RemoveSnapshots { + /// Snapshot ids to remove. + snapshot_ids: Vec, + }, + /// Remove snapshot reference + #[serde(rename_all = "kebab-case")] + RemoveSnapshotRef { + /// Name of snapshot reference to remove. + ref_name: String, + }, + /// Update table's location + SetLocation { + /// New location for table. + location: String, + }, + /// Update table's properties + SetProperties { + /// Properties to update for table. + updates: HashMap, + }, + /// Remove table's properties + RemoveProperties { + /// Properties to remove + removals: Vec, + }, +} + +#[cfg(test)] +mod tests { + use crate::spec::{ + FormatVersion, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, + SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, + Transform, Type, UnboundPartitionField, UnboundPartitionSpec, + }; + use crate::{NamespaceIdent, TableIdent, TableRequirement, TableUpdate}; + use serde::de::DeserializeOwned; + use serde::Serialize; + use std::collections::HashMap; + use std::fmt::Debug; + use uuid::uuid; + + #[test] + fn test_create_table_id() { + let table_id = TableIdent { + namespace: NamespaceIdent::from_strs(vec!["ns1"]).unwrap(), + name: "t1".to_string(), + }; + + assert_eq!(table_id, TableIdent::from_strs(vec!["ns1", "t1"]).unwrap()); + } + + fn test_serde_json( + json: impl ToString, + expected: T, + ) { + let json_str = json.to_string(); + let actual: T = serde_json::from_str(&json_str).expect("Failed to parse from json"); + assert_eq!(actual, expected, "Parsed value is not equal to expected"); + + let restored: T = serde_json::from_str( + &serde_json::to_string(&actual).expect("Failed to serialize to json"), + ) + .expect("Failed to parse from serialized json"); + + assert_eq!( + restored, expected, + "Parsed restored value is not equal to expected" + ); + } + + #[test] + fn test_table_uuid() { + test_serde_json( + r#" +{ + "type": "assert-table-uuid", + "uuid": "2cc52516-5e73-41f2-b139-545d41a4e151" +} + "#, + TableRequirement::UuidMatch { + uuid: uuid!("2cc52516-5e73-41f2-b139-545d41a4e151"), + }, + ); + } + + #[test] + fn test_assert_table_not_exists() { + test_serde_json( + r#" +{ + "type": "assert-create" +} + "#, + TableRequirement::NotExist, + ); + } + + #[test] + fn test_assert_ref_snapshot_id() { + test_serde_json( + r#" +{ + "type": "assert-ref-snapshot-id", + "ref": "snapshot-name", + "snapshot-id": null +} + "#, + TableRequirement::RefSnapshotIdMatch { + r#ref: "snapshot-name".to_string(), + snapshot_id: None, + }, + ); + + test_serde_json( + r#" +{ + "type": "assert-ref-snapshot-id", + "ref": "snapshot-name", + "snapshot-id": 1 +} + "#, + TableRequirement::RefSnapshotIdMatch { + r#ref: "snapshot-name".to_string(), + snapshot_id: Some(1), + }, + ); + } + + #[test] + fn test_assert_last_assigned_field_id() { + test_serde_json( + r#" +{ + "type": "assert-last-assigned-field-id", + "last-assigned-field-id": 12 +} + "#, + TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id: 12, + }, + ); + } + + #[test] + fn test_assert_current_schema_id() { + test_serde_json( + r#" +{ + "type": "assert-current-schema-id", + "current-schema-id": 4 +} + "#, + TableRequirement::CurrentSchemaIdMatch { + current_schema_id: 4, + }, + ); + } + + #[test] + fn test_assert_last_assigned_partition_id() { + test_serde_json( + r#" +{ + "type": "assert-last-assigned-partition-id", + "last-assigned-partition-id": 1004 +} + "#, + TableRequirement::LastAssignedPartitionIdMatch { + last_assigned_partition_id: 1004, + }, + ); + } + + #[test] + fn test_assert_default_spec_id() { + test_serde_json( + r#" +{ + "type": "assert-default-spec-id", + "default-spec-id": 5 +} + "#, + TableRequirement::DefaultSpecIdMatch { default_spec_id: 5 }, + ); + } + + #[test] + fn test_assert_default_sort_order() { + let json = r#" +{ + "type": "assert-default-sort-order-id", + "default-sort-order-id": 10 +} + "#; + + let update = TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: 10, + }; + + test_serde_json(json, update); + } + + #[test] + fn test_parse_assert_invalid() { + assert!( + serde_json::from_str::( + r#" +{ + "default-sort-order-id": 10 +} +"# + ) + .is_err(), + "Table requirements should not be parsed without type." + ); + } + + #[test] + fn test_assign_uuid() { + test_serde_json( + r#" +{ + "action": "assign-uuid", + "uuid": "2cc52516-5e73-41f2-b139-545d41a4e151" +} + "#, + TableUpdate::AssignUuid { + uuid: uuid!("2cc52516-5e73-41f2-b139-545d41a4e151"), + }, + ); + } + + #[test] + fn test_upgrade_format_version() { + test_serde_json( + r#" +{ + "action": "upgrade-format-version", + "format-version": 2 +} + "#, + TableUpdate::UpgradeFormatVersion { + format_version: FormatVersion::V2, + }, + ); + } + + #[test] + fn test_add_schema() { + let test_schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .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(), + ]) + .build() + .unwrap(); + test_serde_json( + r#" +{ + "action": "add-schema", + "schema": { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "foo", + "required": false, + "type": "string" + }, + { + "id": 2, + "name": "bar", + "required": true, + "type": "int" + }, + { + "id": 3, + "name": "baz", + "required": false, + "type": "boolean" + } + ], + "identifier-field-ids": [ + 2 + ] + }, + "last-column-id": 3 +} + "#, + TableUpdate::AddSchema { + schema: test_schema.clone(), + last_column_id: Some(3), + }, + ); + + test_serde_json( + r#" +{ + "action": "add-schema", + "schema": { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "foo", + "required": false, + "type": "string" + }, + { + "id": 2, + "name": "bar", + "required": true, + "type": "int" + }, + { + "id": 3, + "name": "baz", + "required": false, + "type": "boolean" + } + ], + "identifier-field-ids": [ + 2 + ] + } +} + "#, + TableUpdate::AddSchema { + schema: test_schema.clone(), + last_column_id: None, + }, + ); + } + + #[test] + fn test_set_current_schema() { + test_serde_json( + r#" +{ + "action": "set-current-schema", + "schema-id": 23 +} + "#, + TableUpdate::SetCurrentSchema { schema_id: 23 }, + ); + } + + #[test] + fn test_add_spec() { + test_serde_json( + r#" +{ + "action": "add-spec", + "spec": { + "fields": [ + { + "source-id": 4, + "name": "ts_day", + "transform": "day" + }, + { + "source-id": 1, + "name": "id_bucket", + "transform": "bucket[16]" + }, + { + "source-id": 2, + "name": "id_truncate", + "transform": "truncate[4]" + } + ] + } +} + "#, + TableUpdate::AddSpec { + spec: UnboundPartitionSpec::builder() + .with_unbound_partition_field( + UnboundPartitionField::builder() + .source_id(4) + .name("ts_day".to_string()) + .transform(Transform::Day) + .build(), + ) + .with_unbound_partition_field( + UnboundPartitionField::builder() + .source_id(1) + .name("id_bucket".to_string()) + .transform(Transform::Bucket(16)) + .build(), + ) + .with_unbound_partition_field( + UnboundPartitionField::builder() + .source_id(2) + .name("id_truncate".to_string()) + .transform(Transform::Truncate(4)) + .build(), + ) + .build() + .unwrap(), + }, + ); + } + + #[test] + fn test_set_default_spec() { + test_serde_json( + r#" +{ + "action": "set-default-spec", + "spec-id": 1 +} + "#, + TableUpdate::SetDefaultSpec { spec_id: 1 }, + ) + } + + #[test] + fn test_add_sort_order() { + let json = r#" +{ + "action": "add-sort-order", + "sort-order": { + "order-id": 1, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } +} + "#; + + let update = TableUpdate::AddSortOrder { + sort_order: SortOrder::builder() + .with_order_id(1) + .with_sort_field( + SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(), + ) + .with_sort_field( + SortField::builder() + .source_id(3) + .direction(SortDirection::Descending) + .null_order(NullOrder::Last) + .transform(Transform::Bucket(4)) + .build(), + ) + .build_unbound() + .unwrap(), + }; + + test_serde_json(json, update); + } + + #[test] + fn test_set_default_order() { + let json = r#" +{ + "action": "set-default-sort-order", + "sort-order-id": 2 +} + "#; + let update = TableUpdate::SetDefaultSortOrder { sort_order_id: 2 }; + + test_serde_json(json, update); + } + + #[test] + fn test_add_snapshot() { + let json = r#" +{ + "action": "add-snapshot", + "snapshot": { + "snapshot-id": 3055729675574597000, + "parent-snapshot-id": 3051729675574597000, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + } +} + "#; + + let update = TableUpdate::AddSnapshot { + snapshot: Snapshot::builder() + .with_snapshot_id(3055729675574597000) + .with_parent_snapshot_id(Some(3051729675574597000)) + .with_timestamp_ms(1555100955770) + .with_sequence_number(1) + .with_manifest_list("s3://a/b/2.avro") + .with_schema_id(1) + .with_summary(Summary { + operation: Operation::Append, + other: HashMap::default(), + }) + .build(), + }; + + test_serde_json(json, update); + } + + #[test] + fn test_remove_snapshots() { + let json = r#" +{ + "action": "remove-snapshots", + "snapshot-ids": [ + 1, + 2 + ] +} + "#; + + let update = TableUpdate::RemoveSnapshots { + snapshot_ids: vec![1, 2], + }; + test_serde_json(json, update); + } + + #[test] + fn test_remove_snapshot_ref() { + let json = r#" +{ + "action": "remove-snapshot-ref", + "ref-name": "snapshot-ref" +} + "#; + + let update = TableUpdate::RemoveSnapshotRef { + ref_name: "snapshot-ref".to_string(), + }; + test_serde_json(json, update); + } + + #[test] + fn test_set_snapshot_ref_tag() { + let json = r#" +{ + "action": "set-snapshot-ref", + "type": "tag", + "ref-name": "hank", + "snapshot-id": 1, + "max-ref-age-ms": 1 +} + "#; + + let update = TableUpdate::SetSnapshotRef { + ref_name: "hank".to_string(), + reference: SnapshotReference { + snapshot_id: 1, + retention: SnapshotRetention::Tag { max_ref_age_ms: 1 }, + }, + }; + + test_serde_json(json, update); + } + + #[test] + fn test_set_snapshot_ref_branch() { + let json = r#" +{ + "action": "set-snapshot-ref", + "type": "branch", + "ref-name": "hank", + "snapshot-id": 1, + "min-snapshots-to-keep": 2, + "max-snapshot-age-ms": 3, + "max-ref-age-ms": 4 +} + "#; + + let update = TableUpdate::SetSnapshotRef { + ref_name: "hank".to_string(), + reference: SnapshotReference { + snapshot_id: 1, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: Some(2), + max_snapshot_age_ms: Some(3), + max_ref_age_ms: Some(4), + }, + }, + }; + + test_serde_json(json, update); + } + + #[test] + fn test_set_properties() { + let json = r#" +{ + "action": "set-properties", + "updates": { + "prop1": "v1", + "prop2": "v2" + } +} + "#; + + let update = TableUpdate::SetProperties { + updates: vec![ + ("prop1".to_string(), "v1".to_string()), + ("prop2".to_string(), "v2".to_string()), + ] + .into_iter() + .collect(), + }; + + test_serde_json(json, update); + } + + #[test] + fn test_remove_properties() { + let json = r#" +{ + "action": "remove-properties", + "removals": [ + "prop1", + "prop2" + ] +} + "#; + + let update = TableUpdate::RemoveProperties { + removals: vec!["prop1".to_string(), "prop2".to_string()], + }; + + test_serde_json(json, update); + } + + #[test] + fn test_set_location() { + let json = r#" +{ + "action": "set-location", + "location": "s3://bucket/warehouse/tbl_location" +} + "#; + + let update = TableUpdate::SetLocation { + location: "s3://bucket/warehouse/tbl_location".to_string(), + }; + + test_serde_json(json, update); + } +} diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs new file mode 100644 index 0000000..c851402 --- /dev/null +++ b/crates/iceberg/src/error.rs @@ -0,0 +1,429 @@ +// 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::backtrace::{Backtrace, BacktraceStatus}; +use std::fmt; +use std::fmt::Debug; +use std::fmt::Display; +use std::fmt::Formatter; + +/// Result that is a wrapper of `Result` +pub type Result = std::result::Result; + +/// ErrorKind is all kinds of Error of iceberg. +#[derive(Clone, Copy, Debug, PartialEq, Eq)] +#[non_exhaustive] +pub enum ErrorKind { + /// Iceberg don't know what happened here, and no actions other than + /// just returning it back. For example, iceberg returns an internal + /// service error. + Unexpected, + + /// Iceberg data is invalid. + /// + /// This error is returned when we try to read a table from iceberg but + /// failed to parse it's metadata or data file correctly. + /// + /// The table could be invalid or corrupted. + DataInvalid, + /// Iceberg feature is not supported. + /// + /// This error is returned when given iceberg feature is not supported. + FeatureUnsupported, +} + +impl ErrorKind { + /// Convert self into static str. + pub fn into_static(self) -> &'static str { + self.into() + } +} + +impl From for &'static str { + fn from(v: ErrorKind) -> &'static str { + match v { + ErrorKind::Unexpected => "Unexpected", + ErrorKind::DataInvalid => "DataInvalid", + ErrorKind::FeatureUnsupported => "FeatureUnsupported", + } + } +} + +impl Display for ErrorKind { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.into_static()) + } +} + +/// Error is the error struct returned by all iceberg functions. +/// +/// ## Display +/// +/// Error can be displayed in two ways: +/// +/// - Via `Display`: like `err.to_string()` or `format!("{err}")` +/// +/// Error will be printed in a single line: +/// +/// ```shell +/// Unexpected, context: { path: /path/to/file, called: send_async } => something wrong happened, source: networking error" +/// ``` +/// +/// - Via `Debug`: like `format!("{err:?}")` +/// +/// Error will be printed in multi lines with more details and backtraces (if captured): +/// +/// ```shell +/// Unexpected => something wrong happened +/// +/// Context: +/// path: /path/to/file +/// called: send_async +/// +/// Source: networking error +/// +/// Backtrace: +/// 0: iceberg::error::Error::new +/// at ./src/error.rs:197:24 +/// 1: iceberg::error::tests::generate_error +/// at ./src/error.rs:241:9 +/// 2: iceberg::error::tests::test_error_debug_with_backtrace::{{closure}} +/// at ./src/error.rs:305:41 +/// ... +/// ``` +pub struct Error { + kind: ErrorKind, + message: String, + + context: Vec<(&'static str, String)>, + + source: Option, + backtrace: Backtrace, +} + +impl Display for Error { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.kind)?; + + if !self.context.is_empty() { + write!(f, ", context: {{ ")?; + write!( + f, + "{}", + self.context + .iter() + .map(|(k, v)| format!("{k}: {v}")) + .collect::>() + .join(", ") + )?; + write!(f, " }}")?; + } + + if !self.message.is_empty() { + write!(f, " => {}", self.message)?; + } + + if let Some(source) = &self.source { + write!(f, ", source: {source}")?; + } + + Ok(()) + } +} + +impl Debug for Error { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + // If alternate has been specified, we will print like Debug. + if f.alternate() { + let mut de = f.debug_struct("Error"); + de.field("kind", &self.kind); + de.field("message", &self.message); + de.field("context", &self.context); + de.field("source", &self.source); + de.field("backtrace", &self.backtrace); + return de.finish(); + } + + write!(f, "{}", self.kind)?; + if !self.message.is_empty() { + write!(f, " => {}", self.message)?; + } + writeln!(f)?; + + if !self.context.is_empty() { + writeln!(f)?; + writeln!(f, "Context:")?; + for (k, v) in self.context.iter() { + writeln!(f, " {k}: {v}")?; + } + } + if let Some(source) = &self.source { + writeln!(f)?; + writeln!(f, "Source: {source:#}")?; + } + + if self.backtrace.status() == BacktraceStatus::Captured { + writeln!(f)?; + writeln!(f, "Backtrace:")?; + writeln!(f, "{}", self.backtrace)?; + } + + Ok(()) + } +} + +impl std::error::Error for Error { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + self.source.as_ref().map(|v| v.as_ref()) + } +} + +impl Error { + /// Create a new Error with error kind and message. + pub fn new(kind: ErrorKind, message: impl Into) -> Self { + Self { + kind, + message: message.into(), + context: Vec::default(), + + source: None, + // `Backtrace::capture()` will check if backtrace has been enabled + // internally. It's zero cost if backtrace is disabled. + backtrace: Backtrace::capture(), + } + } + + /// Add more context in error. + pub fn with_context(mut self, key: &'static str, value: impl Into) -> Self { + self.context.push((key, value.into())); + self + } + + /// Set source for error. + /// + /// # Notes + /// + /// If the source has been set, we will raise a panic here. + pub fn with_source(mut self, src: impl Into) -> Self { + debug_assert!(self.source.is_none(), "the source error has been set"); + + self.source = Some(src.into()); + self + } + + /// Set the backtrace for error. + /// + /// This function is served as testing purpose and not intended to be called + /// by users. + #[cfg(test)] + fn with_backtrace(mut self, backtrace: Backtrace) -> Self { + self.backtrace = backtrace; + self + } + + /// Return error's kind. + /// + /// Users can use this method to check error's kind and take actions. + pub fn kind(&self) -> ErrorKind { + self.kind + } + + /// Return error's message. + #[inline] + pub fn message(&self) -> &str { + self.message.as_str() + } +} + +macro_rules! define_from_err { + ($source: path, $error_kind: path, $msg: expr) => { + impl std::convert::From<$source> for crate::error::Error { + fn from(v: $source) -> Self { + Self::new($error_kind, $msg).with_source(v) + } + } + }; +} + +define_from_err!( + std::str::Utf8Error, + ErrorKind::Unexpected, + "handling invalid utf-8 characters" +); + +define_from_err!( + std::array::TryFromSliceError, + ErrorKind::DataInvalid, + "failed to convert byte slive to array" +); + +define_from_err!( + std::num::TryFromIntError, + ErrorKind::DataInvalid, + "failed to convert integer" +); + +define_from_err!( + chrono::ParseError, + ErrorKind::DataInvalid, + "Failed to parse string to date or time" +); + +define_from_err!( + uuid::Error, + ErrorKind::DataInvalid, + "Failed to convert between uuid und iceberg value" +); + +define_from_err!( + apache_avro::Error, + ErrorKind::DataInvalid, + "Failure in conversion with avro" +); + +define_from_err!( + opendal::Error, + ErrorKind::Unexpected, + "Failure in doing io operation" +); + +define_from_err!( + url::ParseError, + ErrorKind::DataInvalid, + "Failed to parse url" +); + +define_from_err!( + reqwest::Error, + ErrorKind::Unexpected, + "Failed to execute http request" +); + +define_from_err!( + serde_json::Error, + ErrorKind::DataInvalid, + "Failed to parse json string" +); + +define_from_err!( + rust_decimal::Error, + ErrorKind::DataInvalid, + "Failed to convert decimal literal to rust decimal" +); + +define_from_err!(std::io::Error, ErrorKind::Unexpected, "IO Operation failed"); + +/// Helper macro to check arguments. +/// +/// +/// Example: +/// +/// Following example check `a > 0`, otherwise returns an error. +/// ```ignore +/// use iceberg::check; +/// ensure_data_valid!(a > 0, "{} is not positive.", a); +/// ``` +#[macro_export] +macro_rules! ensure_data_valid { + ($cond: expr, $fmt: literal, $($arg:tt)*) => { + if !$cond { + return Err($crate::error::Error::new($crate::error::ErrorKind::DataInvalid, format!($fmt, $($arg)*))) + } + }; +} + +#[cfg(test)] +mod tests { + use anyhow::anyhow; + use pretty_assertions::assert_eq; + + use super::*; + + fn generate_error_with_backtrace_disabled() -> Error { + Error::new( + ErrorKind::Unexpected, + "something wrong happened".to_string(), + ) + .with_context("path", "/path/to/file".to_string()) + .with_context("called", "send_async".to_string()) + .with_source(anyhow!("networking error")) + .with_backtrace(Backtrace::disabled()) + } + + fn generate_error_with_backtrace_enabled() -> Error { + Error::new( + ErrorKind::Unexpected, + "something wrong happened".to_string(), + ) + .with_context("path", "/path/to/file".to_string()) + .with_context("called", "send_async".to_string()) + .with_source(anyhow!("networking error")) + .with_backtrace(Backtrace::force_capture()) + } + + #[test] + fn test_error_display_without_backtrace() { + let s = format!("{}", generate_error_with_backtrace_disabled()); + assert_eq!( + s, + r#"Unexpected, context: { path: /path/to/file, called: send_async } => something wrong happened, source: networking error"# + ) + } + + #[test] + fn test_error_display_with_backtrace() { + let s = format!("{}", generate_error_with_backtrace_enabled()); + assert_eq!( + s, + r#"Unexpected, context: { path: /path/to/file, called: send_async } => something wrong happened, source: networking error"# + ) + } + + #[test] + fn test_error_debug_without_backtrace() { + let s = format!("{:?}", generate_error_with_backtrace_disabled()); + assert_eq!( + s, + r#"Unexpected => something wrong happened + +Context: + path: /path/to/file + called: send_async + +Source: networking error +"# + ) + } + + /// Backtrace contains build information, so we just assert the header of error content. + #[test] + fn test_error_debug_with_backtrace() { + let s = format!("{:?}", generate_error_with_backtrace_enabled()); + + let expected = r#"Unexpected => something wrong happened + +Context: + path: /path/to/file + called: send_async + +Source: networking error + +Backtrace: + 0:"#; + assert_eq!(&s[..expected.len()], expected,); + } +} diff --git a/crates/iceberg/src/expr/mod.rs b/crates/iceberg/src/expr/mod.rs new file mode 100644 index 0000000..aef1444 --- /dev/null +++ b/crates/iceberg/src/expr/mod.rs @@ -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. + +//! This module contains expressions. + +mod term; +pub use term::*; +mod predicate; +pub use predicate::*; + +/// Predicate operators used in expressions. +#[allow(missing_docs)] +pub enum PredicateOperator { + IsNull, + NotNull, + IsNan, + NotNan, + LessThan, + LessThanOrEq, + GreaterThan, + GreaterThanOrEq, + Eq, + NotEq, + In, + NotIn, + StartsWith, + NotStartsWith, +} diff --git a/crates/iceberg/src/expr/predicate.rs b/crates/iceberg/src/expr/predicate.rs new file mode 100644 index 0000000..9d6bf86 --- /dev/null +++ b/crates/iceberg/src/expr/predicate.rs @@ -0,0 +1,93 @@ +// 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. + +//! This module contains predicate expressions. +//! Predicate expressions are used to filter data, and evaluates to a boolean value. For example, +//! `a > 10` is a predicate expression, and it evaluates to `true` if `a` is greater than `10`, + +use crate::expr::{BoundReference, PredicateOperator, UnboundReference}; +use crate::spec::Literal; +use std::collections::HashSet; + +/// Logical expression, such as `AND`, `OR`, `NOT`. +pub struct LogicalExpression { + inputs: [Box; N], +} + +/// Unary predicate, for example, `a IS NULL`. +pub struct UnaryExpression { + /// Operator of this predicate, must be single operand operator. + op: PredicateOperator, + /// Term of this predicate, for example, `a` in `a IS NULL`. + term: T, +} + +/// Binary predicate, for example, `a > 10`. +pub struct BinaryExpression { + /// Operator of this predicate, must be binary operator, such as `=`, `>`, `<`, etc. + op: PredicateOperator, + /// Term of this predicate, for example, `a` in `a > 10`. + term: T, + /// Literal of this predicate, for example, `10` in `a > 10`. + literal: Literal, +} + +/// Set predicates, for example, `a in (1, 2, 3)`. +pub struct SetExpression { + /// Operator of this predicate, must be set operator, such as `IN`, `NOT IN`, etc. + op: PredicateOperator, + /// Term of this predicate, for example, `a` in `a in (1, 2, 3)`. + term: T, + /// Literals of this predicate, for example, `(1, 2, 3)` in `a in (1, 2, 3)`. + literals: HashSet, +} + +/// Unbound predicate expression before binding to a schema. +pub enum UnboundPredicate { + /// And predicate, for example, `a > 10 AND b < 20`. + And(LogicalExpression), + /// Or predicate, for example, `a > 10 OR b < 20`. + Or(LogicalExpression), + /// Not predicate, for example, `NOT (a > 10)`. + Not(LogicalExpression), + /// Unary expression, for example, `a IS NULL`. + Unary(UnaryExpression), + /// Binary expression, for example, `a > 10`. + Binary(BinaryExpression), + /// Set predicates, for example, `a in (1, 2, 3)`. + Set(SetExpression), +} + +/// Bound predicate expression after binding to a schema. +pub enum BoundPredicate { + /// An expression always evaluates to true. + AlwaysTrue, + /// An expression always evaluates to false. + AlwaysFalse, + /// An expression combined by `AND`, for example, `a > 10 AND b < 20`. + And(LogicalExpression), + /// An expression combined by `OR`, for example, `a > 10 OR b < 20`. + Or(LogicalExpression), + /// An expression combined by `NOT`, for example, `NOT (a > 10)`. + Not(LogicalExpression), + /// Unary expression, for example, `a IS NULL`. + Unary(UnaryExpression), + /// Binary expression, for example, `a > 10`. + Binary(BinaryExpression), + /// Set predicates, for example, `a in (1, 2, 3)`. + Set(SetExpression), +} diff --git a/crates/iceberg/src/expr/term.rs b/crates/iceberg/src/expr/term.rs new file mode 100644 index 0000000..5a81ecd --- /dev/null +++ b/crates/iceberg/src/expr/term.rs @@ -0,0 +1,37 @@ +// 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. + +//! Term definition. + +use crate::spec::NestedFieldRef; + +/// Unbound term before binding to a schema. +pub type UnboundTerm = UnboundReference; + +/// A named reference in an unbound expression. +/// For example, `a` in `a > 10`. +pub struct UnboundReference { + name: String, +} + +/// A named reference in a bound expression after binding to a schema. +pub struct BoundReference { + field: NestedFieldRef, +} + +/// Bound term after binding to a schema. +pub type BoundTerm = BoundReference; diff --git a/crates/iceberg/src/io.rs b/crates/iceberg/src/io.rs new file mode 100644 index 0000000..3a7c85f --- /dev/null +++ b/crates/iceberg/src/io.rs @@ -0,0 +1,508 @@ +// 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. + +//! File io implementation. +//! +//! # How to build `FileIO` +//! +//! We provided a `FileIOBuilder` to build `FileIO` from scratch. For example: +//! ```rust +//! use iceberg::io::{FileIOBuilder, S3_REGION}; +//! +//! let file_io = FileIOBuilder::new("s3") +//! .with_prop(S3_REGION, "us-east-1") +//! .build() +//! .unwrap(); +//! ``` +//! +//! Or you can pass a path to ask `FileIO` to infer schema for you: +//! ```rust +//! use iceberg::io::{FileIO, S3_REGION}; +//! let file_io = FileIO::from_path("s3://bucket/a") +//! .unwrap() +//! .with_prop(S3_REGION, "us-east-1") +//! .build() +//! .unwrap(); +//! ``` +//! +//! # How to use `FileIO` +//! +//! Currently `FileIO` provides simple methods for file operations: +//! +//! - `delete`: Delete file. +//! - `is_exist`: Check if file exists. +//! - `new_input`: Create input file for reading. +//! - `new_output`: Create output file for writing. + +use std::{collections::HashMap, sync::Arc}; + +use crate::{error::Result, Error, ErrorKind}; +use futures::{AsyncRead, AsyncSeek, AsyncWrite}; +use once_cell::sync::Lazy; +use opendal::{Operator, Scheme}; +use url::Url; + +/// Following are arguments for [s3 file io](https://py.iceberg.apache.org/configuration/#s3). +/// S3 endopint. +pub const S3_ENDPOINT: &str = "s3.endpoint"; +/// S3 access key id. +pub const S3_ACCESS_KEY_ID: &str = "s3.access-key-id"; +/// S3 secret access key. +pub const S3_SECRET_ACCESS_KEY: &str = "s3.secret-access-key"; +/// S3 region. +pub const S3_REGION: &str = "s3.region"; + +/// A mapping from iceberg s3 configuration key to [`opendal::Operator`] configuration key. +static S3_CONFIG_MAPPING: Lazy> = Lazy::new(|| { + let mut m = HashMap::with_capacity(4); + m.insert(S3_ENDPOINT, "endpoint"); + m.insert(S3_ACCESS_KEY_ID, "access_key_id"); + m.insert(S3_SECRET_ACCESS_KEY, "secret_access_key"); + m.insert(S3_REGION, "region"); + + m +}); + +const DEFAULT_ROOT_PATH: &str = "/"; + +/// FileIO implementation, used to manipulate files in underlying storage. +/// +/// # Note +/// +/// All path passed to `FileIO` must be absolute path starting with scheme string used to construct `FileIO`. +/// For example, if you construct `FileIO` with `s3a` scheme, then all path passed to `FileIO` must start with `s3a://`. +#[derive(Clone, Debug)] +pub struct FileIO { + inner: Arc, +} + +/// Builder for [`FileIO`]. +#[derive(Debug)] +pub struct FileIOBuilder { + /// This is used to infer scheme of operator. + /// + /// If this is `None`, then [`FileIOBuilder::build`](FileIOBuilder::build) will build a local file io. + scheme_str: Option, + /// Arguments for operator. + props: HashMap, +} + +impl FileIOBuilder { + /// Creates a new builder with scheme. + pub fn new(scheme_str: impl ToString) -> Self { + Self { + scheme_str: Some(scheme_str.to_string()), + props: HashMap::default(), + } + } + + /// Creates a new builder for local file io. + pub fn new_fs_io() -> Self { + Self { + scheme_str: None, + props: HashMap::default(), + } + } + + /// Add argument for operator. + pub fn with_prop(mut self, key: impl ToString, value: impl ToString) -> Self { + self.props.insert(key.to_string(), value.to_string()); + self + } + + /// Add argument for operator. + pub fn with_props( + mut self, + args: impl IntoIterator, + ) -> Self { + self.props + .extend(args.into_iter().map(|e| (e.0.to_string(), e.1.to_string()))); + self + } + + /// Builds [`FileIO`]. + pub fn build(self) -> Result { + let storage = Storage::build(self)?; + Ok(FileIO { + inner: Arc::new(storage), + }) + } +} + +impl FileIO { + /// Try to infer file io scheme from path. + /// + /// If it's a valid url, for example http://example.org, url scheme will be used. + /// If it's not a valid url, will try to detect if it's a file path. + /// + /// Otherwise will return parsing error. + pub fn from_path(path: impl AsRef) -> Result { + let url = Url::parse(path.as_ref()) + .map_err(Error::from) + .or_else(|e| { + Url::from_file_path(path.as_ref()).map_err(|_| { + Error::new( + ErrorKind::DataInvalid, + "Input is neither a valid url nor path", + ) + .with_context("input", path.as_ref().to_string()) + .with_source(e) + }) + })?; + + Ok(FileIOBuilder::new(url.scheme())) + } + + /// Deletes file. + pub async fn delete(&self, path: impl AsRef) -> Result<()> { + let (op, relative_path) = self.inner.create_operator(&path)?; + Ok(op.delete(relative_path).await?) + } + + /// Check file exists. + pub async fn is_exist(&self, path: impl AsRef) -> Result { + let (op, relative_path) = self.inner.create_operator(&path)?; + Ok(op.is_exist(relative_path).await?) + } + + /// Creates input file. + pub fn new_input(&self, path: impl AsRef) -> Result { + let (op, relative_path) = self.inner.create_operator(&path)?; + let path = path.as_ref().to_string(); + let relative_path_pos = path.len() - relative_path.len(); + Ok(InputFile { + op, + path, + relative_path_pos, + }) + } + + /// Creates output file. + pub fn new_output(&self, path: impl AsRef) -> Result { + let (op, relative_path) = self.inner.create_operator(&path)?; + let path = path.as_ref().to_string(); + let relative_path_pos = path.len() - relative_path.len(); + Ok(OutputFile { + op, + path, + relative_path_pos, + }) + } +} + +/// Input file is used for reading from files. +#[derive(Debug)] +pub struct InputFile { + op: Operator, + // Absolution path of file. + path: String, + // Relative path of file to uri, starts at [`relative_path_pos`] + relative_path_pos: usize, +} + +/// Trait for reading file. +pub trait FileRead: AsyncRead + AsyncSeek {} + +impl FileRead for T where T: AsyncRead + AsyncSeek {} + +impl InputFile { + /// Absolute path to root uri. + pub fn location(&self) -> &str { + &self.path + } + + /// Check if file exists. + pub async fn exists(&self) -> Result { + Ok(self + .op + .is_exist(&self.path[self.relative_path_pos..]) + .await?) + } + + /// Creates [`InputStream`] for reading. + pub async fn reader(&self) -> Result { + Ok(self.op.reader(&self.path[self.relative_path_pos..]).await?) + } +} + +/// Trait for writing file. +pub trait FileWrite: AsyncWrite {} + +impl FileWrite for T where T: AsyncWrite {} + +/// Output file is used for writing to files.. +#[derive(Debug)] +pub struct OutputFile { + op: Operator, + // Absolution path of file. + path: String, + // Relative path of file to uri, starts at [`relative_path_pos`] + relative_path_pos: usize, +} + +impl OutputFile { + /// Relative path to root uri. + pub fn location(&self) -> &str { + &self.path + } + + /// Checks if file exists. + pub async fn exists(&self) -> Result { + Ok(self + .op + .is_exist(&self.path[self.relative_path_pos..]) + .await?) + } + + /// Converts into [`InputFile`]. + pub fn to_input_file(self) -> InputFile { + InputFile { + op: self.op, + path: self.path, + relative_path_pos: self.relative_path_pos, + } + } + + /// Creates output file for writing. + pub async fn writer(&self) -> Result { + Ok(self.op.writer(&self.path[self.relative_path_pos..]).await?) + } +} + +// We introduce this because I don't want to handle unsupported `Scheme` in every method. +#[derive(Debug)] +enum Storage { + LocalFs { + op: Operator, + }, + S3 { + scheme_str: String, + props: HashMap, + }, +} + +impl Storage { + /// Creates operator from path. + /// + /// # Arguments + /// + /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + /// + /// # Returns + /// + /// The return value consists of two parts: + /// + /// * An [`opendal::Operator`] instance used to operate on file. + /// * Relative path to the root uri of [`opendal::Operator`]. + /// + fn create_operator<'a>(&self, path: &'a impl AsRef) -> Result<(Operator, &'a str)> { + let path = path.as_ref(); + match self { + Storage::LocalFs { op } => { + if let Some(stripped) = path.strip_prefix("file:/") { + Ok((op.clone(), stripped)) + } else { + Ok((op.clone(), &path[1..])) + } + } + Storage::S3 { scheme_str, props } => { + let mut props = props.clone(); + let url = Url::parse(path)?; + let bucket = url.host_str().ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Invalid s3 url: {}, missing bucket", path), + ) + })?; + + props.insert("bucket".to_string(), bucket.to_string()); + + let prefix = format!("{}://{}/", scheme_str, bucket); + if path.starts_with(&prefix) { + Ok((Operator::via_map(Scheme::S3, props)?, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid s3 url: {}, should start with {}", path, prefix), + )) + } + } + } + } + + /// Parse scheme. + fn parse_scheme(scheme: &str) -> Result { + match scheme { + "file" | "" => Ok(Scheme::Fs), + "s3" | "s3a" => Ok(Scheme::S3), + s => Ok(s.parse::()?), + } + } + + /// Convert iceberg config to opendal config. + fn build(file_io_builder: FileIOBuilder) -> Result { + let scheme_str = file_io_builder.scheme_str.unwrap_or("".to_string()); + let scheme = Self::parse_scheme(&scheme_str)?; + let mut new_props = HashMap::default(); + new_props.insert("root".to_string(), DEFAULT_ROOT_PATH.to_string()); + + match scheme { + Scheme::Fs => Ok(Self::LocalFs { + op: Operator::via_map(Scheme::Fs, new_props)?, + }), + Scheme::S3 => { + for prop in file_io_builder.props { + if let Some(op_key) = S3_CONFIG_MAPPING.get(prop.0.as_str()) { + new_props.insert(op_key.to_string(), prop.1); + } + } + + Ok(Self::S3 { + scheme_str, + props: new_props, + }) + } + _ => Err(Error::new( + ErrorKind::FeatureUnsupported, + format!("Constructing file io from scheme: {scheme} not supported now",), + )), + } + } +} + +#[cfg(test)] +mod tests { + use std::io::Write; + + use std::{fs::File, path::Path}; + + use futures::io::AllowStdIo; + use futures::{AsyncReadExt, AsyncWriteExt}; + + use tempfile::TempDir; + + use super::{FileIO, FileIOBuilder}; + + fn create_local_file_io() -> FileIO { + FileIOBuilder::new_fs_io().build().unwrap() + } + + fn write_to_file>(s: &str, path: P) { + let mut f = File::create(path).unwrap(); + write!(f, "{s}").unwrap(); + } + + async fn read_from_file>(path: P) -> String { + let mut f = AllowStdIo::new(File::open(path).unwrap()); + let mut s = String::new(); + f.read_to_string(&mut s).await.unwrap(); + s + } + + #[tokio::test] + async fn test_local_input_file() { + let tmp_dir = TempDir::new().unwrap(); + + let file_name = "a.txt"; + let content = "Iceberg loves rust."; + + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + write_to_file(content, &full_path); + + let file_io = create_local_file_io(); + let input_file = file_io.new_input(&full_path).unwrap(); + + assert!(input_file.exists().await.unwrap()); + // Remove heading slash + assert_eq!(&full_path, input_file.location()); + let read_content = read_from_file(full_path).await; + + assert_eq!(content, &read_content); + } + + #[tokio::test] + async fn test_delete_local_file() { + let tmp_dir = TempDir::new().unwrap(); + + let file_name = "a.txt"; + let content = "Iceberg loves rust."; + + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + write_to_file(content, &full_path); + + let file_io = create_local_file_io(); + assert!(file_io.is_exist(&full_path).await.unwrap()); + file_io.delete(&full_path).await.unwrap(); + assert!(!file_io.is_exist(&full_path).await.unwrap()); + } + + #[tokio::test] + async fn test_delete_non_exist_file() { + let tmp_dir = TempDir::new().unwrap(); + + let file_name = "a.txt"; + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + + let file_io = create_local_file_io(); + assert!(!file_io.is_exist(&full_path).await.unwrap()); + assert!(file_io.delete(&full_path).await.is_ok()); + } + + #[tokio::test] + async fn test_local_output_file() { + let tmp_dir = TempDir::new().unwrap(); + + let file_name = "a.txt"; + let content = "Iceberg loves rust."; + + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + + let file_io = create_local_file_io(); + let output_file = file_io.new_output(&full_path).unwrap(); + + assert!(!output_file.exists().await.unwrap()); + { + let mut writer = output_file.writer().await.unwrap(); + writer.write_all(content.as_bytes()).await.unwrap(); + writer.close().await.unwrap(); + } + + assert_eq!(&full_path, output_file.location()); + + let read_content = read_from_file(full_path).await; + + assert_eq!(content, &read_content); + } + + #[test] + fn test_create_file_from_path() { + let io = FileIO::from_path("/tmp/a").unwrap(); + assert_eq!("file", io.scheme_str.unwrap().as_str()); + + let io = FileIO::from_path("file:/tmp/b").unwrap(); + assert_eq!("file", io.scheme_str.unwrap().as_str()); + + let io = FileIO::from_path("file:///tmp/c").unwrap(); + assert_eq!("file", io.scheme_str.unwrap().as_str()); + + let io = FileIO::from_path("s3://bucket/a").unwrap(); + assert_eq!("s3", io.scheme_str.unwrap().as_str()); + + let io = FileIO::from_path("tmp/||c"); + assert!(io.is_err()); + } +} diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs new file mode 100644 index 0000000..9ceadca --- /dev/null +++ b/crates/iceberg/src/lib.rs @@ -0,0 +1,55 @@ +// 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. + +//! Native Rust implementation of Apache Iceberg + +#![deny(missing_docs)] + +#[macro_use] +extern crate derive_builder; + +mod error; +pub use error::Error; +pub use error::ErrorKind; +pub use error::Result; + +mod catalog; + +pub use catalog::Catalog; +pub use catalog::Namespace; +pub use catalog::NamespaceIdent; +pub use catalog::TableCommit; +pub use catalog::TableCreation; +pub use catalog::TableIdent; +pub use catalog::TableRequirement; +pub use catalog::TableUpdate; + +#[allow(dead_code)] +pub mod table; + +mod avro; +pub mod io; +pub mod spec; + +mod scan; + +#[allow(dead_code)] +pub mod expr; +pub mod transaction; +pub mod transform; + +pub mod writer; diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs new file mode 100644 index 0000000..0a3b9a9 --- /dev/null +++ b/crates/iceberg/src/scan.rs @@ -0,0 +1,448 @@ +// 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. + +//! Table scan api. + +use crate::io::FileIO; +use crate::spec::{DataContentType, ManifestEntryRef, SchemaRef, SnapshotRef, TableMetadataRef}; +use crate::table::Table; +use crate::{Error, ErrorKind}; +use arrow_array::RecordBatch; +use futures::stream::{iter, BoxStream}; +use futures::StreamExt; + +/// Builder to create table scan. +pub struct TableScanBuilder<'a> { + table: &'a Table, + // Empty column names means to select all columns + column_names: Vec, + snapshot_id: Option, +} + +impl<'a> TableScanBuilder<'a> { + pub fn new(table: &'a Table) -> Self { + Self { + table, + column_names: vec![], + snapshot_id: None, + } + } + + /// Select all columns. + pub fn select_all(mut self) -> Self { + self.column_names.clear(); + self + } + + /// Select some columns of the table. + pub fn select(mut self, column_names: impl IntoIterator) -> Self { + self.column_names = column_names + .into_iter() + .map(|item| item.to_string()) + .collect(); + self + } + + /// Set the snapshot to scan. When not set, it uses current snapshot. + pub fn snapshot_id(mut self, snapshot_id: i64) -> Self { + self.snapshot_id = Some(snapshot_id); + self + } + + /// Build the table scan. + pub fn build(self) -> crate::Result { + let snapshot = match self.snapshot_id { + Some(snapshot_id) => self + .table + .metadata() + .snapshot_by_id(snapshot_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Snapshot with id {} not found", snapshot_id), + ) + })? + .clone(), + None => self + .table + .metadata() + .current_snapshot() + .ok_or_else(|| { + Error::new( + ErrorKind::FeatureUnsupported, + "Can't scan table without snapshots", + ) + })? + .clone(), + }; + + let schema = snapshot.schema(self.table.metadata())?; + + // Check that all column names exist in the schema. + if !self.column_names.is_empty() { + for column_name in &self.column_names { + if schema.field_by_name(column_name).is_none() { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Column {} not found in table.", column_name), + )); + } + } + } + + Ok(TableScan { + snapshot, + file_io: self.table.file_io().clone(), + table_metadata: self.table.metadata_ref(), + column_names: self.column_names, + schema, + }) + } +} + +/// Table scan. +#[derive(Debug)] +#[allow(dead_code)] +pub struct TableScan { + snapshot: SnapshotRef, + table_metadata: TableMetadataRef, + file_io: FileIO, + column_names: Vec, + schema: SchemaRef, +} + +/// A stream of [`FileScanTask`]. +pub type FileScanTaskStream = BoxStream<'static, crate::Result>; + +impl TableScan { + /// Returns a stream of file scan tasks. + pub async fn plan_files(&self) -> crate::Result { + let manifest_list = self + .snapshot + .load_manifest_list(&self.file_io, &self.table_metadata) + .await?; + + // Generate data file stream + let mut file_scan_tasks = Vec::with_capacity(manifest_list.entries().len()); + for manifest_list_entry in manifest_list.entries().iter() { + // Data file + let manifest = manifest_list_entry.load_manifest(&self.file_io).await?; + + for manifest_entry in manifest.entries().iter().filter(|e| e.is_alive()) { + match manifest_entry.content_type() { + DataContentType::EqualityDeletes | DataContentType::PositionDeletes => { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Delete files are not supported yet.", + )); + } + DataContentType::Data => { + file_scan_tasks.push(Ok(FileScanTask { + data_file: manifest_entry.clone(), + start: 0, + length: manifest_entry.file_size_in_bytes(), + })); + } + } + } + } + + Ok(iter(file_scan_tasks).boxed()) + } +} + +/// A task to scan part of file. +#[derive(Debug)] +#[allow(dead_code)] +pub struct FileScanTask { + data_file: ManifestEntryRef, + start: u64, + length: u64, +} + +/// A stream of arrow record batches. +pub type ArrowRecordBatchStream = BoxStream<'static, crate::Result>; + +impl FileScanTask { + /// Returns a stream of arrow record batches. + pub async fn execute(&self) -> crate::Result { + todo!() + } +} + +#[cfg(test)] +mod tests { + use crate::io::{FileIO, OutputFile}; + use crate::spec::{ + DataContentType, DataFile, DataFileFormat, FormatVersion, Literal, Manifest, + ManifestContentType, ManifestEntry, ManifestListWriter, ManifestMetadata, ManifestStatus, + ManifestWriter, Struct, TableMetadata, EMPTY_SNAPSHOT_ID, + }; + use crate::table::Table; + use crate::TableIdent; + use futures::TryStreamExt; + use std::fs; + use tempfile::TempDir; + use tera::{Context, Tera}; + use uuid::Uuid; + + struct TableTestFixture { + table_location: String, + table: Table, + } + + impl TableTestFixture { + fn new() -> Self { + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().join("table1"); + let manifest_list1_location = table_location.join("metadata/manifests_list_1.avro"); + let manifest_list2_location = table_location.join("metadata/manifests_list_2.avro"); + let table_metadata1_location = table_location.join("metadata/v1.json"); + + let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) + .unwrap() + .build() + .unwrap(); + + let table_metadata = { + let template_json_str = fs::read_to_string(format!( + "{}/testdata/example_table_metadata_v2.json", + env!("CARGO_MANIFEST_DIR") + )) + .unwrap(); + let mut context = Context::new(); + context.insert("table_location", &table_location); + context.insert("manifest_list_1_location", &manifest_list1_location); + context.insert("manifest_list_2_location", &manifest_list2_location); + context.insert("table_metadata_1_location", &table_metadata1_location); + + let metadata_json = Tera::one_off(&template_json_str, &context, false).unwrap(); + serde_json::from_str::(&metadata_json).unwrap() + }; + + let table = Table::builder() + .metadata(table_metadata) + .identifier(TableIdent::from_strs(["db", "table1"]).unwrap()) + .file_io(file_io) + .metadata_location(table_metadata1_location.as_os_str().to_str().unwrap()) + .build(); + + Self { + table_location: table_location.to_str().unwrap().to_string(), + table, + } + } + + fn next_manifest_file(&self) -> OutputFile { + self.table + .file_io() + .new_output(format!( + "{}/metadata/manifest_{}.avro", + self.table_location, + Uuid::new_v4() + )) + .unwrap() + } + } + + #[test] + fn test_table_scan_columns() { + let table = TableTestFixture::new().table; + + let table_scan = table.scan().select(["x", "y"]).build().unwrap(); + assert_eq!(vec!["x", "y"], table_scan.column_names); + + let table_scan = table + .scan() + .select(["x", "y"]) + .select(["z"]) + .build() + .unwrap(); + assert_eq!(vec!["z"], table_scan.column_names); + } + + #[test] + fn test_select_all() { + let table = TableTestFixture::new().table; + + let table_scan = table.scan().select_all().build().unwrap(); + assert!(table_scan.column_names.is_empty()); + } + + #[test] + fn test_select_no_exist_column() { + let table = TableTestFixture::new().table; + + let table_scan = table.scan().select(["x", "y", "z", "a"]).build(); + assert!(table_scan.is_err()); + } + + #[test] + fn test_table_scan_default_snapshot_id() { + let table = TableTestFixture::new().table; + + let table_scan = table.scan().build().unwrap(); + assert_eq!( + table.metadata().current_snapshot().unwrap().snapshot_id(), + table_scan.snapshot.snapshot_id() + ); + } + + #[test] + fn test_table_scan_non_exist_snapshot_id() { + let table = TableTestFixture::new().table; + + let table_scan = table.scan().snapshot_id(1024).build(); + assert!(table_scan.is_err()); + } + + #[test] + fn test_table_scan_with_snapshot_id() { + let table = TableTestFixture::new().table; + + let table_scan = table + .scan() + .snapshot_id(3051729675574597004) + .build() + .unwrap(); + assert_eq!(table_scan.snapshot.snapshot_id(), 3051729675574597004); + } + + #[tokio::test] + async fn test_plan_files_no_deletions() { + let fixture = TableTestFixture::new(); + + let current_snapshot = fixture.table.metadata().current_snapshot().unwrap(); + let parent_snapshot = current_snapshot + .parent_snapshot(fixture.table.metadata()) + .unwrap(); + let current_schema = current_snapshot.schema(fixture.table.metadata()).unwrap(); + let current_partition_spec = fixture.table.metadata().default_partition_spec().unwrap(); + + // Write data files + let data_file_manifest = ManifestWriter::new( + fixture.next_manifest_file(), + current_snapshot.snapshot_id(), + vec![], + ) + .write(Manifest::new( + ManifestMetadata::builder() + .schema((*current_schema).clone()) + .content(ManifestContentType::Data) + .format_version(FormatVersion::V2) + .partition_spec((**current_partition_spec).clone()) + .schema_id(current_schema.schema_id()) + .build(), + vec![ + ManifestEntry::builder() + .status(ManifestStatus::Added) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/1.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(100))])) + .build(), + ) + .build(), + ManifestEntry::builder() + .status(ManifestStatus::Deleted) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/2.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(200))])) + .build(), + ) + .build(), + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(parent_snapshot.snapshot_id()) + .sequence_number(parent_snapshot.sequence_number()) + .file_sequence_number(parent_snapshot.sequence_number()) + .data_file( + DataFile::builder() + .content(DataContentType::Data) + .file_path(format!("{}/3.parquet", &fixture.table_location)) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build(), + ) + .build(), + ], + )) + .await + .unwrap(); + + // Write to manifest list + let mut manifest_list_write = ManifestListWriter::v2( + fixture + .table + .file_io() + .new_output(current_snapshot.manifest_list()) + .unwrap(), + current_snapshot.snapshot_id(), + current_snapshot + .parent_snapshot_id() + .unwrap_or(EMPTY_SNAPSHOT_ID), + current_snapshot.sequence_number(), + ); + manifest_list_write + .add_manifest_entries(vec![data_file_manifest].into_iter()) + .unwrap(); + manifest_list_write.close().await.unwrap(); + + // Create table scan for current snapshot and plan files + let table_scan = fixture.table.scan().build().unwrap(); + let mut tasks = table_scan + .plan_files() + .await + .unwrap() + .try_fold(vec![], |mut acc, task| async move { + acc.push(task); + Ok(acc) + }) + .await + .unwrap(); + + assert_eq!(tasks.len(), 2); + + tasks.sort_by_key(|t| t.data_file.file_path().to_string()); + + // Check first task is added data file + assert_eq!( + tasks[0].data_file.file_path(), + format!("{}/1.parquet", &fixture.table_location) + ); + + // Check second task is existing data file + assert_eq!( + tasks[1].data_file.file_path(), + format!("{}/3.parquet", &fixture.table_location) + ); + } +} diff --git a/crates/iceberg/src/spec/datatypes.rs b/crates/iceberg/src/spec/datatypes.rs new file mode 100644 index 0000000..172cb64 --- /dev/null +++ b/crates/iceberg/src/spec/datatypes.rs @@ -0,0 +1,1056 @@ +// 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. + +/*! + * Data Types +*/ +use crate::ensure_data_valid; +use crate::error::Result; +use crate::spec::datatypes::_decimal::{MAX_PRECISION, REQUIRED_LENGTH}; +use ::serde::de::{MapAccess, Visitor}; +use serde::de::{Error, IntoDeserializer}; +use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; +use serde_json::Value as JsonValue; +use std::convert::identity; +use std::sync::Arc; +use std::sync::OnceLock; +use std::{collections::HashMap, fmt, ops::Index}; + +use super::values::Literal; + +/// Field name for list type. +pub(crate) const LIST_FILED_NAME: &str = "element"; +pub(crate) const MAP_KEY_FIELD_NAME: &str = "key"; +pub(crate) const MAP_VALUE_FIELD_NAME: &str = "value"; + +pub(crate) const MAX_DECIMAL_BYTES: u32 = 24; +pub(crate) const MAX_DECIMAL_PRECISION: u32 = 38; + +mod _decimal { + use lazy_static::lazy_static; + + use crate::spec::{MAX_DECIMAL_BYTES, MAX_DECIMAL_PRECISION}; + + lazy_static! { + // Max precision of bytes, starts from 1 + pub(super) static ref MAX_PRECISION: [u32; MAX_DECIMAL_BYTES as usize] = { + let mut ret: [u32; 24] = [0; 24]; + for (i, prec) in ret.iter_mut().enumerate() { + *prec = 2f64.powi((8 * (i + 1) - 1) as i32).log10().floor() as u32; + } + + ret + }; + + // Required bytes of precision, starts from 1 + pub(super) static ref REQUIRED_LENGTH: [u32; MAX_DECIMAL_PRECISION as usize] = { + let mut ret: [u32; MAX_DECIMAL_PRECISION as usize] = [0; MAX_DECIMAL_PRECISION as usize]; + + for (i, required_len) in ret.iter_mut().enumerate() { + for j in 0..MAX_PRECISION.len() { + if MAX_PRECISION[j] >= ((i+1) as u32) { + *required_len = (j+1) as u32; + break; + } + } + } + + ret + }; + + } +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// All data types are either primitives or nested types, which are maps, lists, or structs. +pub enum Type { + /// Primitive types + Primitive(PrimitiveType), + /// Struct type + Struct(StructType), + /// List type. + List(ListType), + /// Map type + Map(MapType), +} + +impl fmt::Display for Type { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + Type::Primitive(primitive) => write!(f, "{}", primitive), + Type::Struct(s) => write!(f, "{}", s), + Type::List(_) => write!(f, "list"), + Type::Map(_) => write!(f, "map"), + } + } +} + +impl Type { + /// Whether the type is primitive type. + #[inline(always)] + pub fn is_primitive(&self) -> bool { + matches!(self, Type::Primitive(_)) + } + + /// Whether the type is struct type. + #[inline(always)] + pub fn is_struct(&self) -> bool { + matches!(self, Type::Struct(_)) + } + + /// Return max precision for decimal given [`num_bytes`] bytes. + #[inline(always)] + pub fn decimal_max_precision(num_bytes: u32) -> Result { + ensure_data_valid!( + num_bytes > 0 && num_bytes <= MAX_DECIMAL_BYTES, + "Decimal length larger than {MAX_DECIMAL_BYTES} is not supported: {num_bytes}", + ); + Ok(MAX_PRECISION[num_bytes as usize - 1]) + } + + /// Returns minimum bytes required for decimal with [`precision`]. + #[inline(always)] + pub fn decimal_required_bytes(precision: u32) -> Result { + ensure_data_valid!(precision > 0 && precision <= MAX_DECIMAL_PRECISION, "Decimals with precision larger than {MAX_DECIMAL_PRECISION} are not supported: {precision}",); + Ok(REQUIRED_LENGTH[precision as usize - 1]) + } + + /// Creates decimal type. + #[inline(always)] + pub fn decimal(precision: u32, scale: u32) -> Result { + ensure_data_valid!(precision > 0 && precision <= MAX_DECIMAL_PRECISION, "Decimals with precision larger than {MAX_DECIMAL_PRECISION} are not supported: {precision}",); + Ok(Type::Primitive(PrimitiveType::Decimal { precision, scale })) + } +} + +impl From for Type { + fn from(value: PrimitiveType) -> Self { + Self::Primitive(value) + } +} + +impl From for Type { + fn from(value: StructType) -> Self { + Type::Struct(value) + } +} + +impl From for Type { + fn from(value: ListType) -> Self { + Type::List(value) + } +} + +impl From for Type { + fn from(value: MapType) -> Self { + Type::Map(value) + } +} + +/// Primitive data types +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "lowercase", remote = "Self")] +pub enum PrimitiveType { + /// True or False + Boolean, + /// 32-bit signed integer + Int, + /// 64-bit signed integer + Long, + /// 32-bit IEEE 754 floating bit. + Float, + /// 64-bit IEEE 754 floating bit. + Double, + /// Fixed point decimal + Decimal { + /// Precision + precision: u32, + /// Scale + scale: u32, + }, + /// Calendar date without timezone or time. + Date, + /// Time of day without date or timezone. + Time, + /// Timestamp without timezone + Timestamp, + /// Timestamp with timezone + Timestamptz, + /// Arbitrary-length character sequences encoded in utf-8 + String, + /// Universally Unique Identifiers + Uuid, + /// Fixed length byte array + Fixed(u64), + /// Arbitrary-length byte array. + Binary, +} + +impl Serialize for Type { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: Serializer, + { + let type_serde = _serde::SerdeType::from(self); + type_serde.serialize(serializer) + } +} + +impl<'de> Deserialize<'de> for Type { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let type_serde = _serde::SerdeType::deserialize(deserializer)?; + Ok(Type::from(type_serde)) + } +} + +impl<'de> Deserialize<'de> for PrimitiveType { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + if s.starts_with("decimal") { + deserialize_decimal(s.into_deserializer()) + } else if s.starts_with("fixed") { + deserialize_fixed(s.into_deserializer()) + } else { + PrimitiveType::deserialize(s.into_deserializer()) + } + } +} + +impl Serialize for PrimitiveType { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: Serializer, + { + match self { + PrimitiveType::Decimal { precision, scale } => { + serialize_decimal(precision, scale, serializer) + } + PrimitiveType::Fixed(l) => serialize_fixed(l, serializer), + _ => PrimitiveType::serialize(self, serializer), + } + } +} + +fn deserialize_decimal<'de, D>(deserializer: D) -> std::result::Result +where + D: Deserializer<'de>, +{ + let s = String::deserialize(deserializer)?; + let (precision, scale) = s + .trim_start_matches(r"decimal(") + .trim_end_matches(')') + .split_once(',') + .ok_or_else(|| D::Error::custom("Decimal requires precision and scale: {s}"))?; + + Ok(PrimitiveType::Decimal { + precision: precision.trim().parse().map_err(D::Error::custom)?, + scale: scale.trim().parse().map_err(D::Error::custom)?, + }) +} + +fn serialize_decimal( + precision: &u32, + scale: &u32, + serializer: S, +) -> std::result::Result +where + S: Serializer, +{ + serializer.serialize_str(&format!("decimal({precision},{scale})")) +} + +fn deserialize_fixed<'de, D>(deserializer: D) -> std::result::Result +where + D: Deserializer<'de>, +{ + let fixed = String::deserialize(deserializer)? + .trim_start_matches(r"fixed[") + .trim_end_matches(']') + .to_owned(); + + fixed + .parse() + .map(PrimitiveType::Fixed) + .map_err(D::Error::custom) +} + +fn serialize_fixed(value: &u64, serializer: S) -> std::result::Result +where + S: Serializer, +{ + serializer.serialize_str(&format!("fixed[{value}]")) +} + +impl fmt::Display for PrimitiveType { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + PrimitiveType::Boolean => write!(f, "boolean"), + PrimitiveType::Int => write!(f, "int"), + PrimitiveType::Long => write!(f, "long"), + PrimitiveType::Float => write!(f, "float"), + PrimitiveType::Double => write!(f, "double"), + PrimitiveType::Decimal { precision, scale } => { + write!(f, "decimal({},{})", precision, scale) + } + PrimitiveType::Date => write!(f, "date"), + PrimitiveType::Time => write!(f, "time"), + PrimitiveType::Timestamp => write!(f, "timestamp"), + PrimitiveType::Timestamptz => write!(f, "timestamptz"), + PrimitiveType::String => write!(f, "string"), + PrimitiveType::Uuid => write!(f, "uuid"), + PrimitiveType::Fixed(size) => write!(f, "fixed({})", size), + PrimitiveType::Binary => write!(f, "binary"), + } + } +} + +/// DataType for a specific struct +#[derive(Debug, Serialize, Clone)] +#[serde(rename = "struct", tag = "type")] +pub struct StructType { + /// Struct fields + fields: Vec, + /// Lookup for index by field id + #[serde(skip_serializing)] + id_lookup: OnceLock>, + #[serde(skip_serializing)] + name_lookup: OnceLock>, +} + +impl<'de> Deserialize<'de> for StructType { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + #[derive(Deserialize)] + #[serde(field_identifier, rename_all = "lowercase")] + enum Field { + Type, + Fields, + } + + struct StructTypeVisitor; + + impl<'de> Visitor<'de> for StructTypeVisitor { + type Value = StructType; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("struct") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: MapAccess<'de>, + { + let mut fields = None; + while let Some(key) = map.next_key()? { + match key { + Field::Type => (), + Field::Fields => { + if fields.is_some() { + return Err(serde::de::Error::duplicate_field("fields")); + } + fields = Some(map.next_value()?); + } + } + } + let fields: Vec = + fields.ok_or_else(|| de::Error::missing_field("fields"))?; + + Ok(StructType::new(fields)) + } + } + + const FIELDS: &[&str] = &["type", "fields"]; + deserializer.deserialize_struct("struct", FIELDS, StructTypeVisitor) + } +} + +impl StructType { + /// Creates a struct type with the given fields. + pub fn new(fields: Vec) -> Self { + Self { + fields, + id_lookup: OnceLock::new(), + name_lookup: OnceLock::new(), + } + } + + /// Get struct field with certain id + pub fn field_by_id(&self, id: i32) -> Option<&NestedFieldRef> { + self.field_id_to_index(id).map(|idx| &self.fields[idx]) + } + + fn field_id_to_index(&self, field_id: i32) -> Option { + self.id_lookup + .get_or_init(|| { + HashMap::from_iter(self.fields.iter().enumerate().map(|(i, x)| (x.id, i))) + }) + .get(&field_id) + .copied() + } + + /// Get struct field with certain field name + pub fn field_by_name(&self, name: &str) -> Option<&NestedFieldRef> { + self.field_name_to_index(name).map(|idx| &self.fields[idx]) + } + + fn field_name_to_index(&self, name: &str) -> Option { + self.name_lookup + .get_or_init(|| { + HashMap::from_iter( + self.fields + .iter() + .enumerate() + .map(|(i, x)| (x.name.clone(), i)), + ) + }) + .get(name) + .copied() + } + + /// Get fields. + pub fn fields(&self) -> &[NestedFieldRef] { + &self.fields + } +} + +impl PartialEq for StructType { + fn eq(&self, other: &Self) -> bool { + self.fields == other.fields + } +} + +impl Eq for StructType {} + +impl Index for StructType { + type Output = NestedField; + + fn index(&self, index: usize) -> &Self::Output { + &self.fields[index] + } +} + +impl fmt::Display for StructType { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "struct<")?; + for field in &self.fields { + write!(f, "{}", field.field_type)?; + } + write!(f, ">") + } +} + +#[derive(Debug, PartialEq, Serialize, Deserialize, Eq, Clone)] +#[serde(from = "SerdeNestedField", into = "SerdeNestedField")] +/// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. +/// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. +/// Fields may have an optional comment or doc string. Fields can have default values. +pub struct NestedField { + /// Id unique in table schema + pub id: i32, + /// Field Name + pub name: String, + /// Optional or required + pub required: bool, + /// Datatype + pub field_type: Box, + /// Fields may have an optional comment or doc string. + pub doc: Option, + /// Used to populate the field’s value for all records that were written before the field was added to the schema + pub initial_default: Option, + /// Used to populate the field’s value for any records written after the field was added to the schema, if the writer does not supply the field’s value + pub write_default: Option, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +struct SerdeNestedField { + pub id: i32, + pub name: String, + pub required: bool, + #[serde(rename = "type")] + pub field_type: Box, + #[serde(skip_serializing_if = "Option::is_none")] + pub doc: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub initial_default: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub write_default: Option, +} + +impl From for NestedField { + fn from(value: SerdeNestedField) -> Self { + NestedField { + id: value.id, + name: value.name, + required: value.required, + initial_default: value.initial_default.and_then(|x| { + Literal::try_from_json(x, &value.field_type) + .ok() + .and_then(identity) + }), + write_default: value.write_default.and_then(|x| { + Literal::try_from_json(x, &value.field_type) + .ok() + .and_then(identity) + }), + field_type: value.field_type, + doc: value.doc, + } + } +} + +impl From for SerdeNestedField { + fn from(value: NestedField) -> Self { + let initial_default = value.initial_default.map(|x| x.try_into_json(&value.field_type).expect("We should have checked this in NestedField::with_initial_default, it can't be converted to json value")); + let write_default = value.write_default.map(|x| x.try_into_json(&value.field_type).expect("We should have checked this in NestedField::with_write_default, it can't be converted to json value")); + SerdeNestedField { + id: value.id, + name: value.name, + required: value.required, + field_type: value.field_type, + doc: value.doc, + initial_default, + write_default, + } + } +} + +/// Reference to nested field. +pub type NestedFieldRef = Arc; + +impl NestedField { + /// Construct a required field. + pub fn required(id: i32, name: impl ToString, field_type: Type) -> Self { + Self { + id, + name: name.to_string(), + required: true, + field_type: Box::new(field_type), + doc: None, + initial_default: None, + write_default: None, + } + } + + /// Construct an optional field. + pub fn optional(id: i32, name: impl ToString, field_type: Type) -> Self { + Self { + id, + name: name.to_string(), + required: false, + field_type: Box::new(field_type), + doc: None, + initial_default: None, + write_default: None, + } + } + + /// Construct list type's element field. + pub fn list_element(id: i32, field_type: Type, required: bool) -> Self { + if required { + Self::required(id, LIST_FILED_NAME, field_type) + } else { + Self::optional(id, LIST_FILED_NAME, field_type) + } + } + + /// Construct map type's key field. + pub fn map_key_element(id: i32, field_type: Type) -> Self { + Self::required(id, MAP_KEY_FIELD_NAME, field_type) + } + + /// Construct map type's value field. + pub fn map_value_element(id: i32, field_type: Type, required: bool) -> Self { + if required { + Self::required(id, MAP_VALUE_FIELD_NAME, field_type) + } else { + Self::optional(id, MAP_VALUE_FIELD_NAME, field_type) + } + } + + /// Set the field's doc. + pub fn with_doc(mut self, doc: impl ToString) -> Self { + self.doc = Some(doc.to_string()); + self + } + + /// Set the field's initial default value. + pub fn with_initial_default(mut self, value: Literal) -> Self { + self.initial_default = Some(value); + self + } + + /// Set the field's initial default value. + pub fn with_write_default(mut self, value: Literal) -> Self { + self.write_default = Some(value); + self + } +} + +impl fmt::Display for NestedField { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}: ", self.id)?; + write!(f, "{}: ", self.name)?; + if self.required { + write!(f, "required ")?; + } else { + write!(f, "optional ")?; + } + write!(f, "{} ", self.field_type)?; + if let Some(doc) = &self.doc { + write!(f, "{}", doc)?; + } + Ok(()) + } +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// A list is a collection of values with some element type. The element field has an integer id that is unique in the table schema. +/// Elements can be either optional or required. Element types may be any type. +pub struct ListType { + /// Element field of list type. + pub element_field: NestedFieldRef, +} + +/// Module for type serialization/deserialization. +pub(super) mod _serde { + use crate::spec::datatypes::Type::Map; + use crate::spec::datatypes::{ + ListType, MapType, NestedField, NestedFieldRef, PrimitiveType, StructType, Type, + }; + use serde_derive::{Deserialize, Serialize}; + use std::borrow::Cow; + + /// List type for serialization and deserialization + #[derive(Serialize, Deserialize)] + #[serde(untagged)] + pub(super) enum SerdeType<'a> { + #[serde(rename_all = "kebab-case")] + List { + r#type: String, + element_id: i32, + element_required: bool, + element: Cow<'a, Type>, + }, + Struct { + r#type: String, + fields: Cow<'a, Vec>, + }, + #[serde(rename_all = "kebab-case")] + Map { + r#type: String, + key_id: i32, + key: Cow<'a, Type>, + value_id: i32, + value_required: bool, + value: Cow<'a, Type>, + }, + Primitive(PrimitiveType), + } + + impl<'a> From> for Type { + fn from(value: SerdeType) -> Self { + match value { + SerdeType::List { + r#type: _, + element_id, + element_required, + element, + } => Self::List(ListType { + element_field: NestedField::list_element( + element_id, + element.into_owned(), + element_required, + ) + .into(), + }), + SerdeType::Map { + r#type: _, + key_id, + key, + value_id, + value_required, + value, + } => Map(MapType { + key_field: NestedField::map_key_element(key_id, key.into_owned()).into(), + value_field: NestedField::map_value_element( + value_id, + value.into_owned(), + value_required, + ) + .into(), + }), + SerdeType::Struct { r#type: _, fields } => { + Self::Struct(StructType::new(fields.into_owned())) + } + SerdeType::Primitive(p) => Self::Primitive(p), + } + } + } + + impl<'a> From<&'a Type> for SerdeType<'a> { + fn from(value: &'a Type) -> Self { + match value { + Type::List(list) => SerdeType::List { + r#type: "list".to_string(), + element_id: list.element_field.id, + element_required: list.element_field.required, + element: Cow::Borrowed(&list.element_field.field_type), + }, + Type::Map(map) => SerdeType::Map { + r#type: "map".to_string(), + key_id: map.key_field.id, + key: Cow::Borrowed(&map.key_field.field_type), + value_id: map.value_field.id, + value_required: map.value_field.required, + value: Cow::Borrowed(&map.value_field.field_type), + }, + Type::Struct(s) => SerdeType::Struct { + r#type: "struct".to_string(), + fields: Cow::Borrowed(&s.fields), + }, + Type::Primitive(p) => SerdeType::Primitive(p.clone()), + } + } + } +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// A map is a collection of key-value pairs with a key type and a value type. +/// Both the key field and value field each have an integer id that is unique in the table schema. +/// Map keys are required and map values can be either optional or required. +/// Both map keys and map values may be any type, including nested types. +pub struct MapType { + /// Field for key. + pub key_field: NestedFieldRef, + /// Field for value. + pub value_field: NestedFieldRef, +} + +#[cfg(test)] +mod tests { + use pretty_assertions::assert_eq; + use uuid::Uuid; + + use crate::spec::values::PrimitiveLiteral; + + use super::*; + + fn check_type_serde(json: &str, expected_type: Type) { + let desered_type: Type = serde_json::from_str(json).unwrap(); + assert_eq!(desered_type, expected_type); + + let sered_json = serde_json::to_string(&expected_type).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + let raw_json_value = serde_json::from_str::(json).unwrap(); + + assert_eq!(parsed_json_value, raw_json_value); + } + + #[test] + fn decimal() { + let record = r#" + { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "id", + "required": true, + "type": "decimal(9,2)" + } + ] + } + "#; + + check_type_serde( + record, + Type::Struct(StructType { + fields: vec![NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Decimal { + precision: 9, + scale: 2, + }), + ) + .into()], + id_lookup: OnceLock::default(), + name_lookup: OnceLock::default(), + }), + ) + } + + #[test] + fn fixed() { + let record = r#" + { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "id", + "required": true, + "type": "fixed[8]" + } + ] + } + "#; + + check_type_serde( + record, + Type::Struct(StructType { + fields: vec![NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Fixed(8)), + ) + .into()], + id_lookup: OnceLock::default(), + name_lookup: OnceLock::default(), + }), + ) + } + + #[test] + fn struct_type() { + let record = r#" + { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "id", + "required": true, + "type": "uuid", + "initial-default": "0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb", + "write-default": "ec5911be-b0a7-458c-8438-c9a3e53cffae" + }, { + "id": 2, + "name": "data", + "required": false, + "type": "int" + } + ] + } + "#; + + check_type_serde( + record, + Type::Struct(StructType { + fields: vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Uuid)) + .with_initial_default(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb").unwrap(), + ))) + .with_write_default(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("ec5911be-b0a7-458c-8438-c9a3e53cffae").unwrap(), + ))) + .into(), + NestedField::optional(2, "data", Type::Primitive(PrimitiveType::Int)).into(), + ], + id_lookup: HashMap::from([(1, 0), (2, 1)]).into(), + name_lookup: HashMap::from([("id".to_string(), 0), ("data".to_string(), 1)]).into(), + }), + ) + } + + #[test] + fn test_deeply_nested_struct() { + let record = r#" +{ + "type": "struct", + "fields": [ + { + "id": 1, + "name": "id", + "required": true, + "type": "uuid", + "initial-default": "0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb", + "write-default": "ec5911be-b0a7-458c-8438-c9a3e53cffae" + }, + { + "id": 2, + "name": "data", + "required": false, + "type": "int" + }, + { + "id": 3, + "name": "address", + "required": true, + "type": { + "type": "struct", + "fields": [ + { + "id": 4, + "name": "street", + "required": true, + "type": "string" + }, + { + "id": 5, + "name": "province", + "required": false, + "type": "string" + }, + { + "id": 6, + "name": "zip", + "required": true, + "type": "int" + } + ] + } + } + ] +} +"#; + + let struct_type = Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Uuid)) + .with_initial_default(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb").unwrap(), + ))) + .with_write_default(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("ec5911be-b0a7-458c-8438-c9a3e53cffae").unwrap(), + ))) + .into(), + NestedField::optional(2, "data", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required( + 3, + "address", + Type::Struct(StructType::new(vec![ + NestedField::required(4, "street", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::optional(5, "province", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::required(6, "zip", Type::Primitive(PrimitiveType::Int)).into(), + ])), + ) + .into(), + ])); + + check_type_serde(record, struct_type) + } + + #[test] + fn list() { + let record = r#" + { + "type": "list", + "element-id": 3, + "element-required": true, + "element": "string" + } + "#; + + check_type_serde( + record, + Type::List(ListType { + element_field: NestedField::list_element( + 3, + Type::Primitive(PrimitiveType::String), + true, + ) + .into(), + }), + ); + } + + #[test] + fn map() { + let record = r#" + { + "type": "map", + "key-id": 4, + "key": "string", + "value-id": 5, + "value-required": false, + "value": "double" + } + "#; + + check_type_serde( + record, + Type::Map(MapType { + key_field: NestedField::map_key_element(4, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 5, + Type::Primitive(PrimitiveType::Double), + false, + ) + .into(), + }), + ); + } + + #[test] + fn map_int() { + let record = r#" + { + "type": "map", + "key-id": 4, + "key": "int", + "value-id": 5, + "value-required": false, + "value": "string" + } + "#; + + check_type_serde( + record, + Type::Map(MapType { + key_field: NestedField::map_key_element(4, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 5, + Type::Primitive(PrimitiveType::String), + false, + ) + .into(), + }), + ); + } + + #[test] + fn test_decimal_precision() { + let expected_max_precision = [ + 2, 4, 6, 9, 11, 14, 16, 18, 21, 23, 26, 28, 31, 33, 35, 38, 40, 43, 45, 47, 50, 52, 55, + 57, + ]; + for (i, max_precision) in expected_max_precision.iter().enumerate() { + assert_eq!( + *max_precision, + Type::decimal_max_precision(i as u32 + 1).unwrap(), + "Failed calculate max precision for {i}" + ); + } + + assert_eq!(5, Type::decimal_required_bytes(10).unwrap()); + assert_eq!(16, Type::decimal_required_bytes(38).unwrap()); + } +} diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs new file mode 100644 index 0000000..e3c989f --- /dev/null +++ b/crates/iceberg/src/spec/manifest.rs @@ -0,0 +1,2007 @@ +// 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. + +//! Manifest for Iceberg. +use self::_const_schema::{manifest_schema_v1, manifest_schema_v2}; + +use super::{ + FieldSummary, FormatVersion, ManifestContentType, ManifestListEntry, PartitionSpec, Schema, + SchemaId, Struct, INITIAL_SEQUENCE_NUMBER, +}; +use super::{Literal, UNASSIGNED_SEQUENCE_NUMBER}; +use crate::error::Result; +use crate::io::OutputFile; +use crate::spec::PartitionField; +use crate::{Error, ErrorKind}; +use apache_avro::{from_value, to_value, Reader as AvroReader, Writer as AvroWriter}; +use futures::AsyncWriteExt; +use serde_json::to_vec; +use std::cmp::min; +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; +use typed_builder::TypedBuilder; + +/// A manifest contains metadata and a list of entries. +#[derive(Debug, PartialEq, Eq, Clone)] +pub struct Manifest { + metadata: ManifestMetadata, + entries: Vec, +} + +impl Manifest { + /// Parse manifest metadata and entries from bytes of avro file. + pub(crate) fn try_from_avro_bytes(bs: &[u8]) -> Result<(ManifestMetadata, Vec)> { + let reader = AvroReader::new(bs)?; + + // Parse manifest metadata + let meta = reader.user_metadata(); + let metadata = ManifestMetadata::parse(meta)?; + + // Parse manifest entries + let partition_type = metadata.partition_spec.partition_type(&metadata.schema)?; + + let entries = match metadata.format_version { + FormatVersion::V1 => { + let schema = manifest_schema_v1(partition_type.clone())?; + let reader = AvroReader::with_schema(&schema, bs)?; + reader + .into_iter() + .map(|value| { + from_value::<_serde::ManifestEntryV1>(&value?)? + .try_into(&partition_type, &metadata.schema) + }) + .collect::>>()? + } + FormatVersion::V2 => { + let schema = manifest_schema_v2(partition_type.clone())?; + let reader = AvroReader::with_schema(&schema, bs)?; + reader + .into_iter() + .map(|value| { + from_value::<_serde::ManifestEntryV2>(&value?)? + .try_into(&partition_type, &metadata.schema) + }) + .collect::>>()? + } + }; + + Ok((metadata, entries)) + } + + /// Parse manifest from bytes of avro file. + pub fn parse_avro(bs: &[u8]) -> Result { + let (metadata, entries) = Self::try_from_avro_bytes(bs)?; + Ok(Self::new(metadata, entries)) + } + + /// Entries slice. + pub fn entries(&self) -> &[ManifestEntryRef] { + &self.entries + } + + /// Constructor from [`ManifestMetadata`] and [`ManifestEntry`]s. + pub fn new(metadata: ManifestMetadata, entries: Vec) -> Self { + Self { + metadata, + entries: entries.into_iter().map(Arc::new).collect(), + } + } +} + +/// A manifest writer. +pub struct ManifestWriter { + output: OutputFile, + + snapshot_id: i64, + + added_files: u32, + added_rows: u64, + existing_files: u32, + existing_rows: u64, + deleted_files: u32, + deleted_rows: u64, + + min_seq_num: Option, + + key_metadata: Vec, + + field_summary: HashMap, +} + +impl ManifestWriter { + /// Create a new manifest writer. + pub fn new(output: OutputFile, snapshot_id: i64, key_metadata: Vec) -> Self { + Self { + output, + snapshot_id, + added_files: 0, + added_rows: 0, + existing_files: 0, + existing_rows: 0, + deleted_files: 0, + deleted_rows: 0, + min_seq_num: None, + key_metadata, + field_summary: HashMap::new(), + } + } + + fn update_field_summary(&mut self, entry: &ManifestEntry) { + // Update field summary + for (&k, &v) in &entry.data_file.null_value_counts { + let field_summary = self.field_summary.entry(k).or_default(); + if v > 0 { + field_summary.contains_null = true; + } + } + + for (&k, &v) in &entry.data_file.nan_value_counts { + let field_summary = self.field_summary.entry(k).or_default(); + if v > 0 { + field_summary.contains_nan = Some(true); + } + if v == 0 { + field_summary.contains_nan = Some(false); + } + } + + for (&k, v) in &entry.data_file.lower_bounds { + let field_summary = self.field_summary.entry(k).or_default(); + if let Some(cur) = &field_summary.lower_bound { + if v < cur { + field_summary.lower_bound = Some(v.clone()); + } + } else { + field_summary.lower_bound = Some(v.clone()); + } + } + + for (&k, v) in &entry.data_file.upper_bounds { + let field_summary = self.field_summary.entry(k).or_default(); + if let Some(cur) = &field_summary.upper_bound { + if v > cur { + field_summary.upper_bound = Some(v.clone()); + } + } else { + field_summary.upper_bound = Some(v.clone()); + } + } + } + + fn get_field_summary_vec(&mut self, spec_fields: &[PartitionField]) -> Vec { + let mut partition_summary = Vec::with_capacity(self.field_summary.len()); + for field in spec_fields { + let entry = self + .field_summary + .remove(&field.source_id) + .unwrap_or_default(); + partition_summary.push(entry); + } + partition_summary + } + + /// Write a manifest entry. + pub async fn write(mut self, manifest: Manifest) -> Result { + // Create the avro writer + let partition_type = manifest + .metadata + .partition_spec + .partition_type(&manifest.metadata.schema)?; + let table_schema = &manifest.metadata.schema; + let avro_schema = match manifest.metadata.format_version { + FormatVersion::V1 => manifest_schema_v1(partition_type.clone())?, + FormatVersion::V2 => manifest_schema_v2(partition_type.clone())?, + }; + let mut avro_writer = AvroWriter::new(&avro_schema, Vec::new()); + avro_writer.add_user_metadata( + "schema".to_string(), + to_vec(table_schema).map_err(|err| { + Error::new(ErrorKind::DataInvalid, "Fail to serialize table schema") + .with_source(err) + })?, + )?; + avro_writer.add_user_metadata( + "schema-id".to_string(), + table_schema.schema_id().to_string(), + )?; + avro_writer.add_user_metadata( + "partition-spec".to_string(), + to_vec(&manifest.metadata.partition_spec.fields).map_err(|err| { + Error::new(ErrorKind::DataInvalid, "Fail to serialize partition spec") + .with_source(err) + })?, + )?; + avro_writer.add_user_metadata( + "partition-spec-id".to_string(), + manifest.metadata.partition_spec.spec_id.to_string(), + )?; + avro_writer.add_user_metadata( + "format-version".to_string(), + (manifest.metadata.format_version as u8).to_string(), + )?; + if manifest.metadata.format_version == FormatVersion::V2 { + avro_writer + .add_user_metadata("content".to_string(), manifest.metadata.content.to_string())?; + } + + // Write manifest entries + for entry in manifest.entries { + if (entry.status == ManifestStatus::Deleted || entry.status == ManifestStatus::Existing) + && (entry.sequence_number.is_none() || entry.file_sequence_number.is_none()) + { + return Err(Error::new( + ErrorKind::DataInvalid, + "Manifest entry with status Existing or Deleted should have sequence number", + )); + } + + match entry.status { + ManifestStatus::Added => { + self.added_files += 1; + self.added_rows += entry.data_file.record_count; + } + ManifestStatus::Deleted => { + self.deleted_files += 1; + self.deleted_rows += entry.data_file.record_count; + } + ManifestStatus::Existing => { + self.existing_files += 1; + self.existing_rows += entry.data_file.record_count; + } + } + + if entry.is_alive() { + if let Some(seq_num) = entry.sequence_number { + self.min_seq_num = Some(self.min_seq_num.map_or(seq_num, |v| min(v, seq_num))); + } + } + + self.update_field_summary(&entry); + + let value = match manifest.metadata.format_version { + FormatVersion::V1 => to_value(_serde::ManifestEntryV1::try_from( + (*entry).clone(), + &partition_type, + )?)? + .resolve(&avro_schema)?, + FormatVersion::V2 => to_value(_serde::ManifestEntryV2::try_from( + (*entry).clone(), + &partition_type, + )?)? + .resolve(&avro_schema)?, + }; + + avro_writer.append(value)?; + } + + let length = avro_writer.flush()?; + let content = avro_writer.into_inner()?; + let mut writer = self.output.writer().await?; + writer.write_all(&content).await.map_err(|err| { + Error::new(ErrorKind::Unexpected, "Fail to write Manifest Entry").with_source(err) + })?; + writer.close().await.map_err(|err| { + Error::new(ErrorKind::Unexpected, "Fail to write Manifest Entry").with_source(err) + })?; + + let partition_summary = + self.get_field_summary_vec(&manifest.metadata.partition_spec.fields); + + Ok(ManifestListEntry { + manifest_path: self.output.location().to_string(), + manifest_length: length as i64, + partition_spec_id: manifest.metadata.partition_spec.spec_id, + content: manifest.metadata.content, + // sequence_number and min_sequence_number with UNASSIGNED_SEQUENCE_NUMBER will be replace with + // real sequence number in `ManifestListWriter`. + sequence_number: UNASSIGNED_SEQUENCE_NUMBER, + min_sequence_number: self.min_seq_num.unwrap_or(UNASSIGNED_SEQUENCE_NUMBER), + added_snapshot_id: self.snapshot_id, + added_data_files_count: Some(self.added_files), + existing_data_files_count: Some(self.existing_files), + deleted_data_files_count: Some(self.deleted_files), + added_rows_count: Some(self.added_rows), + existing_rows_count: Some(self.existing_rows), + deleted_rows_count: Some(self.deleted_rows), + partitions: partition_summary, + key_metadata: self.key_metadata, + }) + } +} + +/// This is a helper module that defines the schema field of the manifest list entry. +mod _const_schema { + use std::sync::Arc; + + use apache_avro::Schema as AvroSchema; + use once_cell::sync::Lazy; + + use crate::{ + avro::schema_to_avro_schema, + spec::{ + ListType, MapType, NestedField, NestedFieldRef, PrimitiveType, Schema, StructType, Type, + }, + Error, + }; + + static STATUS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 0, + "status", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + + static SNAPSHOT_ID_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 1, + "snapshot_id", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static SNAPSHOT_ID_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 1, + "snapshot_id", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static SEQUENCE_NUMBER: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 3, + "sequence_number", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static FILE_SEQUENCE_NUMBER: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 4, + "file_sequence_number", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static CONTENT: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 134, + "content", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + + static FILE_PATH: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 100, + "file_path", + Type::Primitive(PrimitiveType::String), + )) + }) + }; + + static FILE_FORMAT: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 101, + "file_format", + Type::Primitive(PrimitiveType::String), + )) + }) + }; + + static RECORD_COUNT: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 103, + "record_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static FILE_SIZE_IN_BYTES: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 104, + "file_size_in_bytes", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + // Deprecated. Always write a default in v1. Do not write in v2. + static BLOCK_SIZE_IN_BYTES: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 105, + "block_size_in_bytes", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + + static COLUMN_SIZES: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 108, + "column_sizes", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 117, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 118, + "value", + Type::Primitive(PrimitiveType::Long), + )), + }), + )) + }) + }; + + static VALUE_COUNTS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 109, + "value_counts", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 119, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 120, + "value", + Type::Primitive(PrimitiveType::Long), + )), + }), + )) + }) + }; + + static NULL_VALUE_COUNTS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 110, + "null_value_counts", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 121, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 122, + "value", + Type::Primitive(PrimitiveType::Long), + )), + }), + )) + }) + }; + + static NAN_VALUE_COUNTS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 137, + "nan_value_counts", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 138, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 139, + "value", + Type::Primitive(PrimitiveType::Long), + )), + }), + )) + }) + }; + + static LOWER_BOUNDS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 125, + "lower_bounds", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 126, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 127, + "value", + Type::Primitive(PrimitiveType::Binary), + )), + }), + )) + }) + }; + + static UPPER_BOUNDS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 128, + "upper_bounds", + Type::Map(MapType { + key_field: Arc::new(NestedField::required( + 129, + "key", + Type::Primitive(PrimitiveType::Int), + )), + value_field: Arc::new(NestedField::required( + 130, + "value", + Type::Primitive(PrimitiveType::Binary), + )), + }), + )) + }) + }; + + static KEY_METADATA: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 131, + "key_metadata", + Type::Primitive(PrimitiveType::Binary), + )) + }) + }; + + static SPLIT_OFFSETS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 132, + "split_offsets", + Type::List(ListType { + element_field: Arc::new(NestedField::required( + 133, + "element", + Type::Primitive(PrimitiveType::Long), + )), + }), + )) + }) + }; + + static EQUALITY_IDS: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 135, + "equality_ids", + Type::List(ListType { + element_field: Arc::new(NestedField::required( + 136, + "element", + Type::Primitive(PrimitiveType::Int), + )), + }), + )) + }) + }; + + static SORT_ORDER_ID: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 140, + "sort_order_id", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + + pub(super) fn manifest_schema_v2(partition_type: StructType) -> Result { + let fields = vec![ + STATUS.clone(), + SNAPSHOT_ID_V2.clone(), + SEQUENCE_NUMBER.clone(), + FILE_SEQUENCE_NUMBER.clone(), + Arc::new(NestedField::required( + 2, + "data_file", + Type::Struct(StructType::new(vec![ + CONTENT.clone(), + FILE_PATH.clone(), + FILE_FORMAT.clone(), + Arc::new(NestedField::required( + 102, + "partition", + Type::Struct(partition_type), + )), + RECORD_COUNT.clone(), + FILE_SIZE_IN_BYTES.clone(), + COLUMN_SIZES.clone(), + VALUE_COUNTS.clone(), + NULL_VALUE_COUNTS.clone(), + NAN_VALUE_COUNTS.clone(), + LOWER_BOUNDS.clone(), + UPPER_BOUNDS.clone(), + KEY_METADATA.clone(), + SPLIT_OFFSETS.clone(), + EQUALITY_IDS.clone(), + SORT_ORDER_ID.clone(), + ])), + )), + ]; + let schema = Schema::builder().with_fields(fields).build()?; + schema_to_avro_schema("manifest_entry", &schema) + } + + pub(super) fn manifest_schema_v1(partition_type: StructType) -> Result { + let fields = vec![ + STATUS.clone(), + SNAPSHOT_ID_V1.clone(), + Arc::new(NestedField::required( + 2, + "data_file", + Type::Struct(StructType::new(vec![ + FILE_PATH.clone(), + FILE_FORMAT.clone(), + Arc::new(NestedField::required( + 102, + "partition", + Type::Struct(partition_type), + )), + RECORD_COUNT.clone(), + FILE_SIZE_IN_BYTES.clone(), + BLOCK_SIZE_IN_BYTES.clone(), + COLUMN_SIZES.clone(), + VALUE_COUNTS.clone(), + NULL_VALUE_COUNTS.clone(), + NAN_VALUE_COUNTS.clone(), + LOWER_BOUNDS.clone(), + UPPER_BOUNDS.clone(), + KEY_METADATA.clone(), + SPLIT_OFFSETS.clone(), + SORT_ORDER_ID.clone(), + ])), + )), + ]; + let schema = Schema::builder().with_fields(fields).build()?; + schema_to_avro_schema("manifest_entry", &schema) + } +} + +/// Meta data of a manifest that is stored in the key-value metadata of the Avro file +#[derive(Debug, PartialEq, Clone, Eq, TypedBuilder)] +pub struct ManifestMetadata { + /// The table schema at the time the manifest + /// was written + schema: Schema, + /// ID of the schema used to write the manifest as a string + schema_id: SchemaId, + /// The partition spec used to write the manifest + partition_spec: PartitionSpec, + /// Table format version number of the manifest as a string + format_version: FormatVersion, + /// Type of content files tracked by the manifest: “data” or “deletes” + content: ManifestContentType, +} + +impl ManifestMetadata { + /// Parse from metadata in avro file. + pub fn parse(meta: &HashMap>) -> Result { + let schema = { + let bs = meta.get("schema").ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "schema is required in manifest metadata but not found", + ) + })?; + serde_json::from_slice::(bs).map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + "Fail to parse schema in manifest metadata", + ) + .with_source(err) + })? + }; + let schema_id: i32 = meta + .get("schema-id") + .map(|bs| { + String::from_utf8_lossy(bs).parse().map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + "Fail to parse schema id in manifest metadata", + ) + .with_source(err) + }) + }) + .transpose()? + .unwrap_or(0); + let partition_spec = { + let fields = { + let bs = meta.get("partition-spec").ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "partition-spec is required in manifest metadata but not found", + ) + })?; + serde_json::from_slice::>(bs).map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + "Fail to parse partition spec in manifest metadata", + ) + .with_source(err) + })? + }; + let spec_id = meta + .get("partition-spec-id") + .map(|bs| { + String::from_utf8_lossy(bs).parse().map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + "Fail to parse partition spec id in manifest metadata", + ) + .with_source(err) + }) + }) + .transpose()? + .unwrap_or(0); + PartitionSpec { spec_id, fields } + }; + let format_version = if let Some(bs) = meta.get("format-version") { + serde_json::from_slice::(bs).map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + "Fail to parse format version in manifest metadata", + ) + .with_source(err) + })? + } else { + FormatVersion::V1 + }; + let content = if let Some(v) = meta.get("content") { + let v = String::from_utf8_lossy(v); + v.parse()? + } else { + ManifestContentType::Data + }; + Ok(ManifestMetadata { + schema, + schema_id, + partition_spec, + format_version, + content, + }) + } +} + +/// Reference to [`ManifestEntry`]. +pub type ManifestEntryRef = Arc; + +/// A manifest is an immutable Avro file that lists data files or delete +/// files, along with each file’s partition data tuple, metrics, and tracking +/// information. +#[derive(Debug, PartialEq, Eq, Clone, TypedBuilder)] +pub struct ManifestEntry { + /// field: 0 + /// + /// Used to track additions and deletions. + status: ManifestStatus, + /// field id: 1 + /// + /// Snapshot id where the file was added, or deleted if status is 2. + /// Inherited when null. + #[builder(default, setter(strip_option))] + snapshot_id: Option, + /// field id: 3 + /// + /// Data sequence number of the file. + /// Inherited when null and status is 1 (added). + #[builder(default, setter(strip_option))] + sequence_number: Option, + /// field id: 4 + /// + /// File sequence number indicating when the file was added. + /// Inherited when null and status is 1 (added). + #[builder(default, setter(strip_option))] + file_sequence_number: Option, + /// field id: 2 + /// + /// File path, partition tuple, metrics, … + data_file: DataFile, +} + +impl ManifestEntry { + /// Check if this manifest entry is deleted. + pub fn is_alive(&self) -> bool { + matches!( + self.status, + ManifestStatus::Added | ManifestStatus::Existing + ) + } + + /// Content type of this manifest entry. + pub fn content_type(&self) -> DataContentType { + self.data_file.content + } + + /// Data file path of this manifest entry. + pub fn file_path(&self) -> &str { + &self.data_file.file_path + } + + /// Inherit data from manifest list, such as snapshot id, sequence number. + pub(crate) fn inherit_data(&mut self, snapshot_entry: &ManifestListEntry) { + if self.snapshot_id.is_none() { + self.snapshot_id = Some(snapshot_entry.added_snapshot_id); + } + + if self.sequence_number.is_none() + && (self.status == ManifestStatus::Added + || snapshot_entry.sequence_number == INITIAL_SEQUENCE_NUMBER) + { + self.sequence_number = Some(snapshot_entry.sequence_number); + } + + if self.file_sequence_number.is_none() + && (self.status == ManifestStatus::Added + || snapshot_entry.sequence_number == INITIAL_SEQUENCE_NUMBER) + { + self.file_sequence_number = Some(snapshot_entry.sequence_number); + } + } + + /// Data sequence number. + #[inline] + pub fn sequence_number(&self) -> Option { + self.sequence_number + } + + /// File size in bytes. + #[inline] + pub fn file_size_in_bytes(&self) -> u64 { + self.data_file.file_size_in_bytes + } +} + +/// Used to track additions and deletions in ManifestEntry. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum ManifestStatus { + /// Value: 0 + Existing = 0, + /// Value: 1 + Added = 1, + /// Value: 2 + /// + /// Deletes are informational only and not used in scans. + Deleted = 2, +} + +impl TryFrom for ManifestStatus { + type Error = Error; + + fn try_from(v: i32) -> Result { + match v { + 0 => Ok(ManifestStatus::Existing), + 1 => Ok(ManifestStatus::Added), + 2 => Ok(ManifestStatus::Deleted), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("manifest status {} is invalid", v), + )), + } + } +} + +/// Data file carries data file path, partition tuple, metrics, … +#[derive(Debug, PartialEq, Clone, Eq, TypedBuilder)] +pub struct DataFile { + /// field id: 134 + /// + /// Type of content stored by the data file: data, equality deletes, + /// or position deletes (all v1 files are data files) + content: DataContentType, + /// field id: 100 + /// + /// Full URI for the file with FS scheme + file_path: String, + /// field id: 101 + /// + /// String file format name, avro, orc or parquet + file_format: DataFileFormat, + /// field id: 102 + /// + /// Partition data tuple, schema based on the partition spec output using + /// partition field ids for the struct field ids + partition: Struct, + /// field id: 103 + /// + /// Number of records in this file + record_count: u64, + /// field id: 104 + /// + /// Total file size in bytes + file_size_in_bytes: u64, + /// field id: 108 + /// key field id: 117 + /// value field id: 118 + /// + /// Map from column id to the total size on disk of all regions that + /// store the column. Does not include bytes necessary to read other + /// columns, like footers. Leave null for row-oriented formats (Avro) + #[builder(default)] + column_sizes: HashMap, + /// field id: 109 + /// key field id: 119 + /// value field id: 120 + /// + /// Map from column id to number of values in the column (including null + /// and NaN values) + #[builder(default)] + value_counts: HashMap, + /// field id: 110 + /// key field id: 121 + /// value field id: 122 + /// + /// Map from column id to number of null values in the column + #[builder(default)] + null_value_counts: HashMap, + /// field id: 137 + /// key field id: 138 + /// value field id: 139 + /// + /// Map from column id to number of NaN values in the column + #[builder(default)] + nan_value_counts: HashMap, + /// field id: 125 + /// key field id: 126 + /// value field id: 127 + /// + /// Map from column id to lower bound in the column serialized as binary. + /// Each value must be less than or equal to all non-null, non-NaN values + /// in the column for the file. + /// + /// Reference: + /// + /// - [Binary single-value serialization](https://iceberg.apache.org/spec/#binary-single-value-serialization) + #[builder(default)] + lower_bounds: HashMap, + /// field id: 128 + /// key field id: 129 + /// value field id: 130 + /// + /// Map from column id to upper bound in the column serialized as binary. + /// Each value must be greater than or equal to all non-null, non-Nan + /// values in the column for the file. + /// + /// Reference: + /// + /// - [Binary single-value serialization](https://iceberg.apache.org/spec/#binary-single-value-serialization) + #[builder(default)] + upper_bounds: HashMap, + /// field id: 131 + /// + /// Implementation-specific key metadata for encryption + #[builder(default)] + key_metadata: Vec, + /// field id: 132 + /// element field id: 133 + /// + /// Split offsets for the data file. For example, all row group offsets + /// in a Parquet file. Must be sorted ascending + #[builder(default)] + split_offsets: Vec, + /// field id: 135 + /// element field id: 136 + /// + /// Field ids used to determine row equality in equality delete files. + /// Required when content is EqualityDeletes and should be null + /// otherwise. Fields with ids listed in this column must be present + /// in the delete file + #[builder(default)] + equality_ids: Vec, + /// field id: 140 + /// + /// ID representing sort order for this file. + /// + /// If sort order ID is missing or unknown, then the order is assumed to + /// be unsorted. Only data files and equality delete files should be + /// written with a non-null order id. Position deletes are required to be + /// sorted by file and position, not a table order, and should set sort + /// order id to null. Readers must ignore sort order id for position + /// delete files. + #[builder(default, setter(strip_option))] + sort_order_id: Option, +} + +/// Type of content stored by the data file: data, equality deletes, or +/// position deletes (all v1 files are data files) +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum DataContentType { + /// value: 0 + Data = 0, + /// value: 1 + PositionDeletes = 1, + /// value: 2 + EqualityDeletes = 2, +} + +impl TryFrom for DataContentType { + type Error = Error; + + fn try_from(v: i32) -> Result { + match v { + 0 => Ok(DataContentType::Data), + 1 => Ok(DataContentType::PositionDeletes), + 2 => Ok(DataContentType::EqualityDeletes), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("data content type {} is invalid", v), + )), + } + } +} + +/// Format of this data. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum DataFileFormat { + /// Avro file format: + Avro, + /// Orc file format: + Orc, + /// Parquet file format: + Parquet, +} + +impl FromStr for DataFileFormat { + type Err = Error; + + fn from_str(s: &str) -> Result { + match s.to_lowercase().as_str() { + "avro" => Ok(Self::Avro), + "orc" => Ok(Self::Orc), + "parquet" => Ok(Self::Parquet), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("Unsupported data file format: {}", s), + )), + } + } +} + +impl ToString for DataFileFormat { + fn to_string(&self) -> String { + match self { + DataFileFormat::Avro => "avro", + DataFileFormat::Orc => "orc", + DataFileFormat::Parquet => "parquet", + } + .to_string() + } +} + +mod _serde { + use std::collections::HashMap; + + use serde_bytes::ByteBuf; + use serde_derive::{Deserialize, Serialize}; + use serde_with::serde_as; + + use crate::spec::Literal; + use crate::spec::RawLiteral; + use crate::spec::Schema; + use crate::spec::Struct; + use crate::spec::StructType; + use crate::spec::Type; + use crate::Error; + use crate::ErrorKind; + + use super::ManifestEntry; + + #[derive(Serialize, Deserialize)] + pub(super) struct ManifestEntryV2 { + status: i32, + snapshot_id: Option, + sequence_number: Option, + file_sequence_number: Option, + data_file: DataFile, + } + + impl ManifestEntryV2 { + pub fn try_from(value: ManifestEntry, partition_type: &StructType) -> Result { + Ok(Self { + status: value.status as i32, + snapshot_id: value.snapshot_id, + sequence_number: value.sequence_number, + file_sequence_number: value.file_sequence_number, + data_file: DataFile::try_from(value.data_file, partition_type, false)?, + }) + } + + pub fn try_into( + self, + partition_type: &StructType, + schema: &Schema, + ) -> Result { + Ok(ManifestEntry { + status: self.status.try_into()?, + snapshot_id: self.snapshot_id, + sequence_number: self.sequence_number, + file_sequence_number: self.file_sequence_number, + data_file: self.data_file.try_into(partition_type, schema)?, + }) + } + } + + #[derive(Serialize, Deserialize)] + pub(super) struct ManifestEntryV1 { + status: i32, + pub snapshot_id: i64, + data_file: DataFile, + } + + impl ManifestEntryV1 { + pub fn try_from(value: ManifestEntry, partition_type: &StructType) -> Result { + Ok(Self { + status: value.status as i32, + snapshot_id: value.snapshot_id.unwrap_or_default(), + data_file: DataFile::try_from(value.data_file, partition_type, true)?, + }) + } + + pub fn try_into( + self, + partition_type: &StructType, + schema: &Schema, + ) -> Result { + Ok(ManifestEntry { + status: self.status.try_into()?, + snapshot_id: Some(self.snapshot_id), + sequence_number: Some(0), + file_sequence_number: Some(0), + data_file: self.data_file.try_into(partition_type, schema)?, + }) + } + } + + #[serde_as] + #[derive(Serialize, Deserialize)] + pub(super) struct DataFile { + #[serde(default)] + content: i32, + file_path: String, + file_format: String, + partition: RawLiteral, + record_count: i64, + file_size_in_bytes: i64, + #[serde(skip_deserializing, skip_serializing_if = "Option::is_none")] + block_size_in_bytes: Option, + column_sizes: Option>, + value_counts: Option>, + null_value_counts: Option>, + nan_value_counts: Option>, + lower_bounds: Option>, + upper_bounds: Option>, + key_metadata: Option, + split_offsets: Option>, + #[serde(default)] + equality_ids: Option>, + sort_order_id: Option, + } + + impl DataFile { + pub fn try_from( + value: super::DataFile, + partition_type: &StructType, + is_version_1: bool, + ) -> Result { + let block_size_in_bytes = if is_version_1 { Some(0) } else { None }; + Ok(Self { + content: value.content as i32, + file_path: value.file_path, + file_format: value.file_format.to_string(), + partition: RawLiteral::try_from( + Literal::Struct(value.partition), + &Type::Struct(partition_type.clone()), + )?, + record_count: value.record_count.try_into()?, + file_size_in_bytes: value.file_size_in_bytes.try_into()?, + block_size_in_bytes, + column_sizes: Some(to_i64_entry(value.column_sizes)?), + value_counts: Some(to_i64_entry(value.value_counts)?), + null_value_counts: Some(to_i64_entry(value.null_value_counts)?), + nan_value_counts: Some(to_i64_entry(value.nan_value_counts)?), + lower_bounds: Some(to_bytes_entry(value.lower_bounds)), + upper_bounds: Some(to_bytes_entry(value.upper_bounds)), + key_metadata: Some(serde_bytes::ByteBuf::from(value.key_metadata)), + split_offsets: Some(value.split_offsets), + equality_ids: Some(value.equality_ids), + sort_order_id: value.sort_order_id, + }) + } + pub fn try_into( + self, + partition_type: &StructType, + schema: &Schema, + ) -> Result { + let partition = self + .partition + .try_into(&Type::Struct(partition_type.clone()))? + .map(|v| { + if let Literal::Struct(v) = v { + Ok(v) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + "partition value is not a struct", + )) + } + }) + .transpose()? + .unwrap_or(Struct::empty()); + Ok(super::DataFile { + content: self.content.try_into()?, + file_path: self.file_path, + file_format: self.file_format.parse()?, + partition, + record_count: self.record_count.try_into()?, + file_size_in_bytes: self.file_size_in_bytes.try_into()?, + column_sizes: self + .column_sizes + .map(parse_i64_entry) + .transpose()? + .unwrap_or_default(), + value_counts: self + .value_counts + .map(parse_i64_entry) + .transpose()? + .unwrap_or_default(), + null_value_counts: self + .null_value_counts + .map(parse_i64_entry) + .transpose()? + .unwrap_or_default(), + nan_value_counts: self + .nan_value_counts + .map(parse_i64_entry) + .transpose()? + .unwrap_or_default(), + lower_bounds: self + .lower_bounds + .map(|v| parse_bytes_entry(v, schema)) + .transpose()? + .unwrap_or_default(), + upper_bounds: self + .upper_bounds + .map(|v| parse_bytes_entry(v, schema)) + .transpose()? + .unwrap_or_default(), + key_metadata: self.key_metadata.map(|v| v.to_vec()).unwrap_or_default(), + split_offsets: self.split_offsets.unwrap_or_default(), + equality_ids: self.equality_ids.unwrap_or_default(), + sort_order_id: self.sort_order_id, + }) + } + } + + #[serde_as] + #[derive(Serialize, Deserialize)] + #[cfg_attr(test, derive(Debug, PartialEq, Eq))] + struct BytesEntry { + key: i32, + value: serde_bytes::ByteBuf, + } + + fn parse_bytes_entry( + v: Vec, + schema: &Schema, + ) -> Result, Error> { + let mut m = HashMap::with_capacity(v.len()); + for entry in v { + // We ignore the entry if the field is not found in the schema, due to schema evolution. + if let Some(field) = schema.field_by_id(entry.key) { + let data_type = &field.field_type; + m.insert(entry.key, Literal::try_from_bytes(&entry.value, data_type)?); + } + } + Ok(m) + } + + fn to_bytes_entry(v: HashMap) -> Vec { + v.into_iter() + .map(|e| BytesEntry { + key: e.0, + value: Into::::into(e.1), + }) + .collect() + } + + #[derive(Serialize, Deserialize)] + #[cfg_attr(test, derive(Debug, PartialEq, Eq))] + struct I64Entry { + key: i32, + value: i64, + } + + fn parse_i64_entry(v: Vec) -> Result, Error> { + let mut m = HashMap::with_capacity(v.len()); + for entry in v { + // We ignore the entry if it's value is negative since these entries are supposed to be used for + // counting, which should never be negative. + if let Ok(v) = entry.value.try_into() { + m.insert(entry.key, v); + } + } + Ok(m) + } + + fn to_i64_entry(entries: HashMap) -> Result, Error> { + entries + .iter() + .map(|e| { + Ok(I64Entry { + key: *e.0, + value: (*e.1).try_into()?, + }) + }) + .collect() + } + + #[cfg(test)] + mod tests { + use crate::spec::manifest::_serde::{parse_i64_entry, I64Entry}; + use std::collections::HashMap; + + #[test] + fn test_parse_negative_manifest_entry() { + let entries = vec![ + I64Entry { key: 1, value: -1 }, + I64Entry { key: 2, value: 3 }, + ]; + + let ret = parse_i64_entry(entries).unwrap(); + + let expected_ret = HashMap::from([(2, 3)]); + assert_eq!(ret, expected_ret, "Negative i64 entry should be ignored!"); + } + } +} + +#[cfg(test)] +mod tests { + use std::fs; + + use tempfile::TempDir; + + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::NestedField; + use crate::spec::PrimitiveType; + use crate::spec::Struct; + use crate::spec::Transform; + use crate::spec::Type; + use std::sync::Arc; + + #[tokio::test] + async fn test_parse_manifest_v2_unpartition() { + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema: Schema::builder() + .with_fields(vec![ + // id v_int v_long v_float v_double v_varchar v_bool v_date v_timestamp v_decimal v_ts_ntz + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 3, + "v_long", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 4, + "v_float", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 5, + "v_double", + Type::Primitive(PrimitiveType::Double), + )), + Arc::new(NestedField::optional( + 6, + "v_varchar", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 7, + "v_bool", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 8, + "v_date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::optional( + 9, + "v_timestamp", + Type::Primitive(PrimitiveType::Timestamptz), + )), + Arc::new(NestedField::optional( + 10, + "v_decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 36, + scale: 10, + }), + )), + Arc::new(NestedField::optional( + 11, + "v_ts_ntz", + Type::Primitive(PrimitiveType::Timestamp), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V2, + }, + entries: vec![ + Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-ba56fbfa-f2ff-40c9-bb27-565ad6dc2be8-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::empty(), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([(0,73),(6,34),(2,73),(7,61),(3,61),(5,62),(9,79),(10,73),(1,61),(4,73),(8,73)]), + value_counts: HashMap::from([(4,1),(5,1),(2,1),(0,1),(3,1),(6,1),(8,1),(1,1),(10,1),(7,1),(9,1)]), + null_value_counts: HashMap::from([(1,0),(6,0),(2,0),(8,0),(0,0),(3,0),(5,0),(9,0),(7,0),(4,0),(10,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: Vec::new(), + split_offsets: vec![4], + equality_ids: Vec::new(), + sort_order_id: None, + } + }) + ] + }; + + let writer = |output_file: OutputFile| ManifestWriter::new(output_file, 1, vec![]); + + test_manifest_read_write(manifest, writer).await; + } + + #[tokio::test] + async fn test_parse_manifest_v2_partition() { + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema: Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 3, + "v_long", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 4, + "v_float", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 5, + "v_double", + Type::Primitive(PrimitiveType::Double), + )), + Arc::new(NestedField::optional( + 6, + "v_varchar", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 7, + "v_bool", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 8, + "v_date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::optional( + 9, + "v_timestamp", + Type::Primitive(PrimitiveType::Timestamptz), + )), + Arc::new(NestedField::optional( + 10, + "v_decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 36, + scale: 10, + }), + )), + Arc::new(NestedField::optional( + 11, + "v_ts_ntz", + Type::Primitive(PrimitiveType::Timestamp), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![ + PartitionField { + name: "v_int".to_string(), + transform: Transform::Identity, + source_id: 2, + field_id: 1000, + }, + PartitionField { + name: "v_long".to_string(), + transform: Transform::Identity, + source_id: 3, + field_id: 1001, + }, + ], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V2, + }, + entries: vec![Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_format: DataFileFormat::Parquet, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-378b56f5-5c52-4102-a2c2-f05f8a7cbe4a-00000.parquet".to_string(), + partition: Struct::from_iter( + vec![ + Some(Literal::int(1)), + Some(Literal::long(1000)), + ] + .into_iter() + ), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([ + (0, 73), + (6, 34), + (2, 73), + (7, 61), + (3, 61), + (5, 62), + (9, 79), + (10, 73), + (1, 61), + (4, 73), + (8, 73) + ]), + value_counts: HashMap::from([ + (4, 1), + (5, 1), + (2, 1), + (0, 1), + (3, 1), + (6, 1), + (8, 1), + (1, 1), + (10, 1), + (7, 1), + (9, 1) + ]), + null_value_counts: HashMap::from([ + (1, 0), + (6, 0), + (2, 0), + (8, 0), + (0, 0), + (3, 0), + (5, 0), + (9, 0), + (7, 0), + (4, 0), + (10, 0) + ]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: vec![], + split_offsets: vec![4], + equality_ids: vec![], + sort_order_id: None, + }, + })], + }; + + let writer = |output_file: OutputFile| ManifestWriter::new(output_file, 1, vec![]); + + let res = test_manifest_read_write(manifest, writer).await; + + assert_eq!(res.sequence_number, UNASSIGNED_SEQUENCE_NUMBER); + assert_eq!(res.min_sequence_number, UNASSIGNED_SEQUENCE_NUMBER); + } + + #[tokio::test] + async fn test_parse_manifest_v1_unpartition() { + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 1, + schema: Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 3, + "comment", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V1, + }, + entries: vec![Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: Some(0), + sequence_number: Some(0), + file_sequence_number: Some(0), + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3://testbucket/iceberg_data/iceberg_ctl/iceberg_db/iceberg_tbl/data/00000-7-45268d71-54eb-476c-b42c-942d880c04a1-00001.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::empty(), + record_count: 1, + file_size_in_bytes: 875, + column_sizes: HashMap::from([(1,47),(2,48),(3,52)]), + value_counts: HashMap::from([(1,1),(2,1),(3,1)]), + null_value_counts: HashMap::from([(1,0),(2,0),(3,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::from([(1,Literal::int(1)),(2,Literal::string("a")),(3,Literal::string("AC/DC"))]), + upper_bounds: HashMap::from([(1,Literal::int(1)),(2,Literal::string("a")),(3,Literal::string("AC/DC"))]), + key_metadata: vec![], + split_offsets: vec![4], + equality_ids: vec![], + sort_order_id: Some(0), + } + })], + }; + + let writer = + |output_file: OutputFile| ManifestWriter::new(output_file, 2966623707104393227, vec![]); + + test_manifest_read_write(manifest, writer).await; + } + + #[tokio::test] + async fn test_parse_manifest_v1_partition() { + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema: Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 3, + "category", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![PartitionField { + name: "category".to_string(), + transform: Transform::Identity, + source_id: 3, + field_id: 1000, + }], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V1, + }, + entries: vec![ + Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: Some(0), + sequence_number: Some(0), + file_sequence_number: Some(0), + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3://testbucket/prod/db/sample/data/category=x/00010-1-d5c93668-1e52-41ac-92a6-bba590cbf249-00001.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::from_iter( + vec![ + Some( + Literal::try_from_bytes(&[120], &Type::Primitive(PrimitiveType::String)) + .unwrap() + ), + ] + .into_iter() + ), + record_count: 1, + file_size_in_bytes: 874, + column_sizes: HashMap::from([(1, 46), (2, 48), (3, 48)]), + value_counts: HashMap::from([(1, 1), (2, 1), (3, 1)]), + null_value_counts: HashMap::from([(1, 0), (2, 0), (3, 0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::string("a")), + (3, Literal::string("x")) + ]), + upper_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::string("a")), + (3, Literal::string("x")) + ]), + key_metadata: vec![], + split_offsets: vec![4], + equality_ids: vec![], + sort_order_id: Some(0), + }, + }) + ] + }; + + let writer = |output_file: OutputFile| ManifestWriter::new(output_file, 1, vec![]); + + let entry = test_manifest_read_write(manifest, writer).await; + + assert_eq!(entry.partitions.len(), 1); + assert_eq!(entry.partitions[0].lower_bound, Some(Literal::string("x"))); + assert_eq!(entry.partitions[0].upper_bound, Some(Literal::string("x"))); + } + + #[tokio::test] + async fn test_parse_manifest_with_schema_evolution() { + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema: Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V2, + }, + entries: vec![Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_format: DataFileFormat::Parquet, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-378b56f5-5c52-4102-a2c2-f05f8a7cbe4a-00000.parquet".to_string(), + partition: Struct::empty(), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([ + (1, 61), + (2, 73), + (3, 61), + ]), + value_counts: HashMap::default(), + null_value_counts: HashMap::default(), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::int(2)), + (3, Literal::string("x")) + ]), + upper_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::int(2)), + (3, Literal::string("x")) + ]), + key_metadata: vec![], + split_offsets: vec![4], + equality_ids: vec![], + sort_order_id: None, + }, + })], + }; + + let writer = |output_file: OutputFile| ManifestWriter::new(output_file, 1, vec![]); + + let (avro_bytes, _) = write_manifest(&manifest, writer).await; + + // The parse should succeed. + let actual_manifest = Manifest::parse_avro(avro_bytes.as_slice()).unwrap(); + + // Compared with original manifest, the lower_bounds and upper_bounds no longer has data for field 3, and + // other parts should be same. + let expected_manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema: Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + ]) + .build() + .unwrap(), + partition_spec: PartitionSpec { + spec_id: 0, + fields: vec![], + }, + content: ManifestContentType::Data, + format_version: FormatVersion::V2, + }, + entries: vec![Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_format: DataFileFormat::Parquet, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-378b56f5-5c52-4102-a2c2-f05f8a7cbe4a-00000.parquet".to_string(), + partition: Struct::empty(), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([ + (1, 61), + (2, 73), + (3, 61), + ]), + value_counts: HashMap::default(), + null_value_counts: HashMap::default(), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::int(2)), + ]), + upper_bounds: HashMap::from([ + (1, Literal::long(1)), + (2, Literal::int(2)), + ]), + key_metadata: vec![], + split_offsets: vec![4], + equality_ids: vec![], + sort_order_id: None, + }, + })], + }; + + assert_eq!(actual_manifest, expected_manifest); + } + + async fn test_manifest_read_write( + manifest: Manifest, + writer_builder: impl FnOnce(OutputFile) -> ManifestWriter, + ) -> ManifestListEntry { + let (bs, res) = write_manifest(&manifest, writer_builder).await; + let actual_manifest = Manifest::parse_avro(bs.as_slice()).unwrap(); + + assert_eq!(actual_manifest, manifest); + res + } + + /// Utility method which writes out a manifest and returns the bytes. + async fn write_manifest( + manifest: &Manifest, + writer_builder: impl FnOnce(OutputFile) -> ManifestWriter, + ) -> (Vec, ManifestListEntry) { + let temp_dir = TempDir::new().unwrap(); + let path = temp_dir.path().join("test_manifest.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(path.to_str().unwrap()).unwrap(); + let writer = writer_builder(output_file); + let res = writer.write(manifest.clone()).await.unwrap(); + + // Verify manifest + (fs::read(path).expect("read_file must succeed"), res) + } +} diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs new file mode 100644 index 0000000..6dc4839 --- /dev/null +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -0,0 +1,1468 @@ +// 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. + +//! ManifestList for Iceberg. + +use std::{collections::HashMap, str::FromStr}; + +use crate::io::FileIO; +use crate::{io::OutputFile, spec::Literal, Error, ErrorKind}; +use apache_avro::{from_value, types::Value, Reader, Writer}; +use futures::{AsyncReadExt, AsyncWriteExt}; + +use self::{ + _const_schema::{MANIFEST_LIST_AVRO_SCHEMA_V1, MANIFEST_LIST_AVRO_SCHEMA_V2}, + _serde::{ManifestListEntryV1, ManifestListEntryV2}, +}; + +use super::{FormatVersion, Manifest, StructType}; +use crate::error::Result; + +/// Placeholder for sequence number. The field with this value must be replaced with the actual sequence number before it write. +pub const UNASSIGNED_SEQUENCE_NUMBER: i64 = -1; + +/// Snapshots are embedded in table metadata, but the list of manifests for a +/// snapshot are stored in a separate manifest list file. +/// +/// A new manifest list is written for each attempt to commit a snapshot +/// because the list of manifests always changes to produce a new snapshot. +/// When a manifest list is written, the (optimistic) sequence number of the +/// snapshot is written for all new manifest files tracked by the list. +/// +/// A manifest list includes summary metadata that can be used to avoid +/// scanning all of the manifests in a snapshot when planning a table scan. +/// This includes the number of added, existing, and deleted files, and a +/// summary of values for each field of the partition spec used to write the +/// manifest. +#[derive(Debug, Clone, PartialEq)] +pub struct ManifestList { + /// Entries in a manifest list. + entries: Vec, +} + +impl ManifestList { + /// Parse manifest list from bytes. + pub fn parse_with_version( + bs: &[u8], + version: FormatVersion, + partition_type_provider: impl Fn(i32) -> Result>, + ) -> Result { + match version { + FormatVersion::V1 => { + let reader = Reader::with_schema(&MANIFEST_LIST_AVRO_SCHEMA_V1, bs)?; + let values = Value::Array(reader.collect::, _>>()?); + from_value::<_serde::ManifestListV1>(&values)?.try_into(partition_type_provider) + } + FormatVersion::V2 => { + let reader = Reader::with_schema(&MANIFEST_LIST_AVRO_SCHEMA_V2, bs)?; + let values = Value::Array(reader.collect::, _>>()?); + from_value::<_serde::ManifestListV2>(&values)?.try_into(partition_type_provider) + } + } + } + + /// Get the entries in the manifest list. + pub fn entries(&self) -> &[ManifestListEntry] { + &self.entries + } +} + +/// A manifest list writer. +pub struct ManifestListWriter { + format_version: FormatVersion, + output_file: OutputFile, + avro_writer: Writer<'static, Vec>, + sequence_number: i64, + snapshot_id: i64, +} + +impl std::fmt::Debug for ManifestListWriter { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ManifestListWriter") + .field("format_version", &self.format_version) + .field("output_file", &self.output_file) + .field("avro_writer", &self.avro_writer.schema()) + .finish_non_exhaustive() + } +} + +impl ManifestListWriter { + /// Construct a v1 [`ManifestListWriter`] that writes to a provided [`OutputFile`]. + pub fn v1(output_file: OutputFile, snapshot_id: i64, parent_snapshot_id: i64) -> Self { + let metadata = HashMap::from_iter([ + ("snapshot-id".to_string(), snapshot_id.to_string()), + ( + "parent-snapshot-id".to_string(), + parent_snapshot_id.to_string(), + ), + ("format-version".to_string(), "1".to_string()), + ]); + Self::new(FormatVersion::V1, output_file, metadata, 0, snapshot_id) + } + + /// Construct a v2 [`ManifestListWriter`] that writes to a provided [`OutputFile`]. + pub fn v2( + output_file: OutputFile, + snapshot_id: i64, + parent_snapshot_id: i64, + sequence_number: i64, + ) -> Self { + let metadata = HashMap::from_iter([ + ("snapshot-id".to_string(), snapshot_id.to_string()), + ( + "parent-snapshot-id".to_string(), + parent_snapshot_id.to_string(), + ), + ("sequence-number".to_string(), sequence_number.to_string()), + ("format-version".to_string(), "2".to_string()), + ]); + Self::new( + FormatVersion::V2, + output_file, + metadata, + sequence_number, + snapshot_id, + ) + } + + fn new( + format_version: FormatVersion, + output_file: OutputFile, + metadata: HashMap, + sequence_number: i64, + snapshot_id: i64, + ) -> Self { + let avro_schema = match format_version { + FormatVersion::V1 => &MANIFEST_LIST_AVRO_SCHEMA_V1, + FormatVersion::V2 => &MANIFEST_LIST_AVRO_SCHEMA_V2, + }; + let mut avro_writer = Writer::new(avro_schema, Vec::new()); + for (key, value) in metadata { + avro_writer + .add_user_metadata(key, value) + .expect("Avro metadata should be added to the writer before the first record."); + } + Self { + format_version, + output_file, + avro_writer, + sequence_number, + snapshot_id, + } + } + + /// Append manifest entries to be written. + pub fn add_manifest_entries( + &mut self, + manifest_entries: impl Iterator, + ) -> Result<()> { + match self.format_version { + FormatVersion::V1 => { + for manifest_entry in manifest_entries { + let manifest_entry: ManifestListEntryV1 = manifest_entry.try_into()?; + self.avro_writer.append_ser(manifest_entry)?; + } + } + FormatVersion::V2 => { + for mut manifest_entry in manifest_entries { + if manifest_entry.sequence_number == UNASSIGNED_SEQUENCE_NUMBER { + if manifest_entry.added_snapshot_id != self.snapshot_id { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Found unassigned sequence number for a manifest from snapshot {}.", + manifest_entry.added_snapshot_id + ), + )); + } + manifest_entry.sequence_number = self.sequence_number; + } + if manifest_entry.min_sequence_number == UNASSIGNED_SEQUENCE_NUMBER { + if manifest_entry.added_snapshot_id != self.snapshot_id { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Found unassigned sequence number for a manifest from snapshot {}.", + manifest_entry.added_snapshot_id + ), + )); + } + manifest_entry.min_sequence_number = self.sequence_number; + } + let manifest_entry: ManifestListEntryV2 = manifest_entry.try_into()?; + self.avro_writer.append_ser(manifest_entry)?; + } + } + } + Ok(()) + } + + /// Write the manifest list to the output file. + pub async fn close(self) -> Result<()> { + let data = self.avro_writer.into_inner()?; + let mut writer = self.output_file.writer().await?; + writer.write_all(&data).await?; + writer.close().await?; + Ok(()) + } +} + +/// This is a helper module that defines the schema field of the manifest list entry. +mod _const_schema { + use std::sync::Arc; + + use apache_avro::Schema as AvroSchema; + use once_cell::sync::Lazy; + + use crate::{ + avro::schema_to_avro_schema, + spec::{ListType, NestedField, NestedFieldRef, PrimitiveType, Schema, StructType, Type}, + }; + + static MANIFEST_PATH: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 500, + "manifest_path", + Type::Primitive(PrimitiveType::String), + )) + }) + }; + static MANIFEST_LENGTH: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 501, + "manifest_length", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static PARTITION_SPEC_ID: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 502, + "partition_spec_id", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static CONTENT: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 517, + "content", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static SEQUENCE_NUMBER: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 515, + "sequence_number", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static MIN_SEQUENCE_NUMBER: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 516, + "min_sequence_number", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static ADDED_SNAPSHOT_ID: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 503, + "added_snapshot_id", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static ADDED_FILES_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 504, + "added_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static ADDED_FILES_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 504, + "added_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static EXISTING_FILES_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 505, + "existing_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static EXISTING_FILES_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 505, + "existing_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static DELETED_FILES_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 506, + "deleted_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static DELETED_FILES_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 506, + "deleted_data_files_count", + Type::Primitive(PrimitiveType::Int), + )) + }) + }; + static ADDED_ROWS_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 512, + "added_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static ADDED_ROWS_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 512, + "added_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static EXISTING_ROWS_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 513, + "existing_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static EXISTING_ROWS_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 513, + "existing_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static DELETED_ROWS_COUNT_V2: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::required( + 514, + "deleted_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static DELETED_ROWS_COUNT_V1: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 514, + "deleted_rows_count", + Type::Primitive(PrimitiveType::Long), + )) + }) + }; + static PARTITIONS: Lazy = { + Lazy::new(|| { + // element type + let fields = vec![ + Arc::new(NestedField::required( + 509, + "contains_null", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 518, + "contains_nan", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 510, + "lower_bound", + Type::Primitive(PrimitiveType::Binary), + )), + Arc::new(NestedField::optional( + 511, + "upper_bound", + Type::Primitive(PrimitiveType::Binary), + )), + ]; + let element_field = Arc::new(NestedField::required( + 508, + "r_508", + Type::Struct(StructType::new(fields)), + )); + Arc::new(NestedField::optional( + 507, + "partitions", + Type::List(ListType { element_field }), + )) + }) + }; + static KEY_METADATA: Lazy = { + Lazy::new(|| { + Arc::new(NestedField::optional( + 519, + "key_metadata", + Type::Primitive(PrimitiveType::Binary), + )) + }) + }; + + static V1_SCHEMA: Lazy = { + Lazy::new(|| { + let fields = vec![ + MANIFEST_PATH.clone(), + MANIFEST_LENGTH.clone(), + PARTITION_SPEC_ID.clone(), + ADDED_SNAPSHOT_ID.clone(), + ADDED_FILES_COUNT_V1.clone().to_owned(), + EXISTING_FILES_COUNT_V1.clone(), + DELETED_FILES_COUNT_V1.clone(), + ADDED_ROWS_COUNT_V1.clone(), + EXISTING_ROWS_COUNT_V1.clone(), + DELETED_ROWS_COUNT_V1.clone(), + PARTITIONS.clone(), + KEY_METADATA.clone(), + ]; + Schema::builder().with_fields(fields).build().unwrap() + }) + }; + + static V2_SCHEMA: Lazy = { + Lazy::new(|| { + let fields = vec![ + MANIFEST_PATH.clone(), + MANIFEST_LENGTH.clone(), + PARTITION_SPEC_ID.clone(), + CONTENT.clone(), + SEQUENCE_NUMBER.clone(), + MIN_SEQUENCE_NUMBER.clone(), + ADDED_SNAPSHOT_ID.clone(), + ADDED_FILES_COUNT_V2.clone(), + EXISTING_FILES_COUNT_V2.clone(), + DELETED_FILES_COUNT_V2.clone(), + ADDED_ROWS_COUNT_V2.clone(), + EXISTING_ROWS_COUNT_V2.clone(), + DELETED_ROWS_COUNT_V2.clone(), + PARTITIONS.clone(), + KEY_METADATA.clone(), + ]; + Schema::builder().with_fields(fields).build().unwrap() + }) + }; + + pub(super) static MANIFEST_LIST_AVRO_SCHEMA_V1: Lazy = + Lazy::new(|| schema_to_avro_schema("manifest_file", &V1_SCHEMA).unwrap()); + + pub(super) static MANIFEST_LIST_AVRO_SCHEMA_V2: Lazy = + Lazy::new(|| schema_to_avro_schema("manifest_file", &V2_SCHEMA).unwrap()); +} + +/// Entry in a manifest list. +#[derive(Debug, PartialEq, Clone)] +pub struct ManifestListEntry { + /// field: 500 + /// + /// Location of the manifest file + pub manifest_path: String, + /// field: 501 + /// + /// Length of the manifest file in bytes + pub manifest_length: i64, + /// field: 502 + /// + /// ID of a partition spec used to write the manifest; must be listed + /// in table metadata partition-specs + pub partition_spec_id: i32, + /// field: 517 + /// + /// The type of files tracked by the manifest, either data or delete + /// files; 0 for all v1 manifests + pub content: ManifestContentType, + /// field: 515 + /// + /// The sequence number when the manifest was added to the table; use 0 + /// when reading v1 manifest lists + pub sequence_number: i64, + /// field: 516 + /// + /// The minimum data sequence number of all live data or delete files in + /// the manifest; use 0 when reading v1 manifest lists + pub min_sequence_number: i64, + /// field: 503 + /// + /// ID of the snapshot where the manifest file was added + pub added_snapshot_id: i64, + /// field: 504 + /// + /// Number of entries in the manifest that have status ADDED, when null + /// this is assumed to be non-zero + pub added_data_files_count: Option, + /// field: 505 + /// + /// Number of entries in the manifest that have status EXISTING (0), + /// when null this is assumed to be non-zero + pub existing_data_files_count: Option, + /// field: 506 + /// + /// Number of entries in the manifest that have status DELETED (2), + /// when null this is assumed to be non-zero + pub deleted_data_files_count: Option, + /// field: 512 + /// + /// Number of rows in all of files in the manifest that have status + /// ADDED, when null this is assumed to be non-zero + pub added_rows_count: Option, + /// field: 513 + /// + /// Number of rows in all of files in the manifest that have status + /// EXISTING, when null this is assumed to be non-zero + pub existing_rows_count: Option, + /// field: 514 + /// + /// Number of rows in all of files in the manifest that have status + /// DELETED, when null this is assumed to be non-zero + pub deleted_rows_count: Option, + /// field: 507 + /// element_field: 508 + /// + /// A list of field summaries for each partition field in the spec. Each + /// field in the list corresponds to a field in the manifest file’s + /// partition spec. + pub partitions: Vec, + /// field: 519 + /// + /// Implementation-specific key metadata for encryption + pub key_metadata: Vec, +} + +/// The type of files tracked by the manifest, either data or delete files; Data(0) for all v1 manifests +#[derive(Debug, PartialEq, Clone, Eq)] +pub enum ManifestContentType { + /// The manifest content is data. + Data = 0, + /// The manifest content is deletes. + Deletes = 1, +} + +impl FromStr for ManifestContentType { + type Err = Error; + + fn from_str(s: &str) -> Result { + match s { + "data" => Ok(ManifestContentType::Data), + "deletes" => Ok(ManifestContentType::Deletes), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid manifest content type: {s}"), + )), + } + } +} + +impl ToString for ManifestContentType { + fn to_string(&self) -> String { + match self { + ManifestContentType::Data => "data".to_string(), + ManifestContentType::Deletes => "deletes".to_string(), + } + } +} + +impl TryFrom for ManifestContentType { + type Error = Error; + + fn try_from(value: i32) -> std::result::Result { + match value { + 0 => Ok(ManifestContentType::Data), + 1 => Ok(ManifestContentType::Deletes), + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + format!( + "Invalid manifest content type. Expected 0 or 1, got {}", + value + ), + )), + } + } +} + +impl ManifestListEntry { + /// Load [`Manifest`]. + /// + /// This method will also initialize inherited values of [`ManifestEntry`], such as `sequence_number`. + pub async fn load_manifest(&self, file_io: &FileIO) -> Result { + let mut avro = Vec::new(); + file_io + .new_input(&self.manifest_path)? + .reader() + .await? + .read_to_end(&mut avro) + .await?; + + let (metadata, mut entries) = Manifest::try_from_avro_bytes(&avro)?; + + // Let entries inherit values from the manifest list entry. + for entry in &mut entries { + entry.inherit_data(self); + } + + Ok(Manifest::new(metadata, entries)) + } +} + +/// Field summary for partition field in the spec. +/// +/// Each field in the list corresponds to a field in the manifest file’s partition spec. +#[derive(Debug, PartialEq, Eq, Clone, Default)] +pub struct FieldSummary { + /// field: 509 + /// + /// Whether the manifest contains at least one partition with a null + /// value for the field + pub contains_null: bool, + /// field: 518 + /// Whether the manifest contains at least one partition with a NaN + /// value for the field + pub contains_nan: Option, + /// field: 510 + /// The minimum value for the field in the manifests + /// partitions. + pub lower_bound: Option, + /// field: 511 + /// The maximum value for the field in the manifests + /// partitions. + pub upper_bound: Option, +} + +/// This is a helper module that defines types to help with serialization/deserialization. +/// For deserialization the input first gets read into either the [ManifestListEntryV1] or [ManifestListEntryV2] struct +/// and then converted into the [ManifestListEntry] struct. Serialization works the other way around. +/// [ManifestListEntryV1] and [ManifestListEntryV2] are internal struct that are only used for serialization and deserialization. +pub(super) mod _serde { + use crate::{ + spec::{Literal, StructType, Type}, + Error, + }; + pub use serde_bytes::ByteBuf; + use serde_derive::{Deserialize, Serialize}; + + use super::ManifestListEntry; + use crate::error::Result; + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(transparent)] + pub(crate) struct ManifestListV2 { + entries: Vec, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(transparent)] + pub(crate) struct ManifestListV1 { + entries: Vec, + } + + impl ManifestListV2 { + /// Converts the [ManifestListV2] into a [ManifestList]. + /// The convert of [entries] need the partition_type info so use this function instead of [std::TryFrom] trait. + pub fn try_into( + self, + partition_type_provider: impl Fn(i32) -> Result>, + ) -> Result { + Ok(super::ManifestList { + entries: self + .entries + .into_iter() + .map(|v| { + let partition_spec_id = v.partition_spec_id; + let manifest_path = v.manifest_path.clone(); + v.try_into(partition_type_provider(partition_spec_id)?.as_ref()) + .map_err(|err| { + err.with_context("manifest file path", manifest_path) + .with_context( + "partition spec id", + partition_spec_id.to_string(), + ) + }) + }) + .collect::>>()?, + }) + } + } + + impl TryFrom for ManifestListV2 { + type Error = Error; + + fn try_from(value: super::ManifestList) -> std::result::Result { + Ok(Self { + entries: value + .entries + .into_iter() + .map(TryInto::try_into) + .collect::, _>>()?, + }) + } + } + + impl ManifestListV1 { + /// Converts the [ManifestListV1] into a [ManifestList]. + /// The convert of [entries] need the partition_type info so use this function instead of [std::TryFrom] trait. + pub fn try_into( + self, + partition_type_provider: impl Fn(i32) -> Result>, + ) -> Result { + Ok(super::ManifestList { + entries: self + .entries + .into_iter() + .map(|v| { + let partition_spec_id = v.partition_spec_id; + let manifest_path = v.manifest_path.clone(); + v.try_into(partition_type_provider(partition_spec_id)?.as_ref()) + .map_err(|err| { + err.with_context("manifest file path", manifest_path) + .with_context( + "partition spec id", + partition_spec_id.to_string(), + ) + }) + }) + .collect::>>()?, + }) + } + } + + impl TryFrom for ManifestListV1 { + type Error = Error; + + fn try_from(value: super::ManifestList) -> std::result::Result { + Ok(Self { + entries: value + .entries + .into_iter() + .map(TryInto::try_into) + .collect::, _>>()?, + }) + } + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + pub(super) struct ManifestListEntryV1 { + pub manifest_path: String, + pub manifest_length: i64, + pub partition_spec_id: i32, + pub added_snapshot_id: i64, + pub added_data_files_count: Option, + pub existing_data_files_count: Option, + pub deleted_data_files_count: Option, + pub added_rows_count: Option, + pub existing_rows_count: Option, + pub deleted_rows_count: Option, + pub partitions: Option>, + pub key_metadata: Option, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + pub(super) struct ManifestListEntryV2 { + pub manifest_path: String, + pub manifest_length: i64, + pub partition_spec_id: i32, + pub content: i32, + pub sequence_number: i64, + pub min_sequence_number: i64, + pub added_snapshot_id: i64, + pub added_data_files_count: i32, + pub existing_data_files_count: i32, + pub deleted_data_files_count: i32, + pub added_rows_count: i64, + pub existing_rows_count: i64, + pub deleted_rows_count: i64, + pub partitions: Option>, + pub key_metadata: Option, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + pub(super) struct FieldSummary { + pub contains_null: bool, + pub contains_nan: Option, + pub lower_bound: Option, + pub upper_bound: Option, + } + + impl FieldSummary { + /// Converts the [FieldSummary] into a [super::FieldSummary]. + /// [lower_bound] and [upper_bound] are converted into [Literal]s need the type info so use + /// this function instead of [std::TryFrom] trait. + pub(crate) fn try_into(self, r#type: &Type) -> Result { + Ok(super::FieldSummary { + contains_null: self.contains_null, + contains_nan: self.contains_nan, + lower_bound: self + .lower_bound + .map(|v| Literal::try_from_bytes(&v, r#type)) + .transpose()?, + upper_bound: self + .upper_bound + .map(|v| Literal::try_from_bytes(&v, r#type)) + .transpose()?, + }) + } + } + + fn try_convert_to_field_summary( + partitions: Option>, + partition_type: Option<&StructType>, + ) -> Result> { + if let Some(partitions) = partitions { + if let Some(partition_type) = partition_type { + let partition_types = partition_type.fields(); + if partitions.len() != partition_types.len() { + return Err(Error::new( + crate::ErrorKind::DataInvalid, + format!( + "Invalid partition spec. Expected {} fields, got {}", + partition_types.len(), + partitions.len() + ), + )); + } + partitions + .into_iter() + .zip(partition_types) + .map(|(v, field)| v.try_into(&field.field_type)) + .collect::>>() + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "Invalid partition spec. Partition type is required", + )) + } + } else { + Ok(Vec::new()) + } + } + + impl ManifestListEntryV2 { + /// Converts the [ManifestListEntryV2] into a [ManifestListEntry]. + /// The convert of [partitions] need the partition_type info so use this function instead of [std::TryFrom] trait. + pub fn try_into(self, partition_type: Option<&StructType>) -> Result { + let partitions = try_convert_to_field_summary(self.partitions, partition_type)?; + Ok(ManifestListEntry { + manifest_path: self.manifest_path, + manifest_length: self.manifest_length, + partition_spec_id: self.partition_spec_id, + content: self.content.try_into()?, + sequence_number: self.sequence_number, + min_sequence_number: self.min_sequence_number, + added_snapshot_id: self.added_snapshot_id, + added_data_files_count: Some(self.added_data_files_count.try_into()?), + existing_data_files_count: Some(self.existing_data_files_count.try_into()?), + deleted_data_files_count: Some(self.deleted_data_files_count.try_into()?), + added_rows_count: Some(self.added_rows_count.try_into()?), + existing_rows_count: Some(self.existing_rows_count.try_into()?), + deleted_rows_count: Some(self.deleted_rows_count.try_into()?), + partitions, + key_metadata: self.key_metadata.map(|b| b.into_vec()).unwrap_or_default(), + }) + } + } + + impl ManifestListEntryV1 { + /// Converts the [ManifestListEntryV1] into a [ManifestListEntry]. + /// The convert of [partitions] need the partition_type info so use this function instead of [std::TryFrom] trait. + pub fn try_into(self, partition_type: Option<&StructType>) -> Result { + let partitions = try_convert_to_field_summary(self.partitions, partition_type)?; + Ok(ManifestListEntry { + manifest_path: self.manifest_path, + manifest_length: self.manifest_length, + partition_spec_id: self.partition_spec_id, + added_snapshot_id: self.added_snapshot_id, + added_data_files_count: self + .added_data_files_count + .map(TryInto::try_into) + .transpose()?, + existing_data_files_count: self + .existing_data_files_count + .map(TryInto::try_into) + .transpose()?, + deleted_data_files_count: self + .deleted_data_files_count + .map(TryInto::try_into) + .transpose()?, + added_rows_count: self.added_rows_count.map(TryInto::try_into).transpose()?, + existing_rows_count: self + .existing_rows_count + .map(TryInto::try_into) + .transpose()?, + deleted_rows_count: self.deleted_rows_count.map(TryInto::try_into).transpose()?, + partitions, + key_metadata: self.key_metadata.map(|b| b.into_vec()).unwrap_or_default(), + // as ref: https://iceberg.apache.org/spec/#partitioning + // use 0 when reading v1 manifest lists + content: super::ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + }) + } + } + + fn convert_to_serde_field_summary( + partitions: Vec, + ) -> Option> { + if partitions.is_empty() { + None + } else { + Some( + partitions + .into_iter() + .map(|v| FieldSummary { + contains_null: v.contains_null, + contains_nan: v.contains_nan, + lower_bound: v.lower_bound.map(|v| v.into()), + upper_bound: v.upper_bound.map(|v| v.into()), + }) + .collect(), + ) + } + } + + fn convert_to_serde_key_metadata(key_metadata: Vec) -> Option { + if key_metadata.is_empty() { + None + } else { + Some(ByteBuf::from(key_metadata)) + } + } + + impl TryFrom for ManifestListEntryV2 { + type Error = Error; + + fn try_from(value: ManifestListEntry) -> std::result::Result { + let partitions = convert_to_serde_field_summary(value.partitions); + let key_metadata = convert_to_serde_key_metadata(value.key_metadata); + Ok(Self { + manifest_path: value.manifest_path, + manifest_length: value.manifest_length, + partition_spec_id: value.partition_spec_id, + content: value.content as i32, + sequence_number: value.sequence_number, + min_sequence_number: value.min_sequence_number, + added_snapshot_id: value.added_snapshot_id, + added_data_files_count: value + .added_data_files_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "added_data_files_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + existing_data_files_count: value + .existing_data_files_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "existing_data_files_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + deleted_data_files_count: value + .deleted_data_files_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "deleted_data_files_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + added_rows_count: value + .added_rows_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "added_rows_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + existing_rows_count: value + .existing_rows_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "existing_rows_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + deleted_rows_count: value + .deleted_rows_count + .ok_or_else(|| { + Error::new( + crate::ErrorKind::DataInvalid, + "deleted_rows_count in ManifestListEntryV2 should be require", + ) + })? + .try_into()?, + partitions, + key_metadata, + }) + } + } + + impl TryFrom for ManifestListEntryV1 { + type Error = Error; + + fn try_from(value: ManifestListEntry) -> std::result::Result { + let partitions = convert_to_serde_field_summary(value.partitions); + let key_metadata = convert_to_serde_key_metadata(value.key_metadata); + Ok(Self { + manifest_path: value.manifest_path, + manifest_length: value.manifest_length, + partition_spec_id: value.partition_spec_id, + added_snapshot_id: value.added_snapshot_id, + added_data_files_count: value + .added_data_files_count + .map(TryInto::try_into) + .transpose()?, + existing_data_files_count: value + .existing_data_files_count + .map(TryInto::try_into) + .transpose()?, + deleted_data_files_count: value + .deleted_data_files_count + .map(TryInto::try_into) + .transpose()?, + added_rows_count: value.added_rows_count.map(TryInto::try_into).transpose()?, + existing_rows_count: value + .existing_rows_count + .map(TryInto::try_into) + .transpose()?, + deleted_rows_count: value + .deleted_rows_count + .map(TryInto::try_into) + .transpose()?, + partitions, + key_metadata, + }) + } + } +} + +#[cfg(test)] +mod test { + use std::{collections::HashMap, fs, sync::Arc}; + + use tempfile::TempDir; + + use crate::{ + io::FileIOBuilder, + spec::{ + manifest_list::{_serde::ManifestListV1, UNASSIGNED_SEQUENCE_NUMBER}, + FieldSummary, Literal, ManifestContentType, ManifestList, ManifestListEntry, + ManifestListWriter, NestedField, PrimitiveType, StructType, Type, + }, + }; + + use super::_serde::ManifestListV2; + + #[tokio::test] + async fn test_parse_manifest_list_v1() { + let manifest_list = ManifestList { + entries: vec![ + ManifestListEntry { + manifest_path: "/opt/bitnami/spark/warehouse/db/table/metadata/10d28031-9739-484c-92db-cdf2975cead4-m0.avro".to_string(), + manifest_length: 5806, + partition_spec_id: 0, + content: ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + added_snapshot_id: 1646658105718557341, + added_data_files_count: Some(3), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![], + key_metadata: vec![], + } + ] + }; + + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + + let tmp_dir = TempDir::new().unwrap(); + let file_name = "simple_manifest_list_v1.avro"; + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + + let mut writer = ManifestListWriter::v1( + file_io.new_output(full_path.clone()).unwrap(), + 1646658105718557341, + 1646658105718557341, + ); + + writer + .add_manifest_entries(manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + + let bs = fs::read(full_path).expect("read_file must succeed"); + + let parsed_manifest_list = + ManifestList::parse_with_version(&bs, crate::spec::FormatVersion::V1, |_id| Ok(None)) + .unwrap(); + + assert_eq!(manifest_list, parsed_manifest_list); + } + + #[tokio::test] + async fn test_parse_manifest_list_v2() { + let manifest_list = ManifestList { + entries: vec![ + ManifestListEntry { + manifest_path: "s3a://icebergdata/demo/s1/t1/metadata/05ffe08b-810f-49b3-a8f4-e88fc99b254a-m0.avro".to_string(), + manifest_length: 6926, + partition_spec_id: 1, + content: ManifestContentType::Data, + sequence_number: 1, + min_sequence_number: 1, + added_snapshot_id: 377075049360453639, + added_data_files_count: Some(1), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::long(1)), upper_bound: Some(Literal::long(1))}], + key_metadata: vec![], + }, + ManifestListEntry { + manifest_path: "s3a://icebergdata/demo/s1/t1/metadata/05ffe08b-810f-49b3-a8f4-e88fc99b254a-m1.avro".to_string(), + manifest_length: 6926, + partition_spec_id: 2, + content: ManifestContentType::Data, + sequence_number: 1, + min_sequence_number: 1, + added_snapshot_id: 377075049360453639, + added_data_files_count: Some(1), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::float(1.1)), upper_bound: Some(Literal::float(2.1))}], + key_metadata: vec![], + } + ] + }; + + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + + let tmp_dir = TempDir::new().unwrap(); + let file_name = "simple_manifest_list_v1.avro"; + let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); + + let mut writer = ManifestListWriter::v2( + file_io.new_output(full_path.clone()).unwrap(), + 1646658105718557341, + 1646658105718557341, + 1, + ); + + writer + .add_manifest_entries(manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + + let bs = fs::read(full_path).expect("read_file must succeed"); + + let parsed_manifest_list = + ManifestList::parse_with_version(&bs, crate::spec::FormatVersion::V2, |id| { + Ok(HashMap::from([ + ( + 1, + StructType::new(vec![Arc::new(NestedField::required( + 1, + "test", + Type::Primitive(PrimitiveType::Long), + ))]), + ), + ( + 2, + StructType::new(vec![Arc::new(NestedField::required( + 1, + "test", + Type::Primitive(PrimitiveType::Float), + ))]), + ), + ]) + .get(&id) + .cloned()) + }) + .unwrap(); + + assert_eq!(manifest_list, parsed_manifest_list); + } + + #[test] + fn test_serialize_manifest_list_v1() { + let manifest_list:ManifestListV1 = ManifestList { + entries: vec![ManifestListEntry { + manifest_path: "/opt/bitnami/spark/warehouse/db/table/metadata/10d28031-9739-484c-92db-cdf2975cead4-m0.avro".to_string(), + manifest_length: 5806, + partition_spec_id: 0, + content: ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + added_snapshot_id: 1646658105718557341, + added_data_files_count: Some(3), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![], + key_metadata: vec![], + }] + }.try_into().unwrap(); + let result = serde_json::to_string(&manifest_list).unwrap(); + assert_eq!( + result, + r#"[{"manifest_path":"/opt/bitnami/spark/warehouse/db/table/metadata/10d28031-9739-484c-92db-cdf2975cead4-m0.avro","manifest_length":5806,"partition_spec_id":0,"added_snapshot_id":1646658105718557341,"added_data_files_count":3,"existing_data_files_count":0,"deleted_data_files_count":0,"added_rows_count":3,"existing_rows_count":0,"deleted_rows_count":0,"partitions":null,"key_metadata":null}]"# + ); + } + + #[test] + fn test_serialize_manifest_list_v2() { + let manifest_list:ManifestListV2 = ManifestList { + entries: vec![ManifestListEntry { + manifest_path: "s3a://icebergdata/demo/s1/t1/metadata/05ffe08b-810f-49b3-a8f4-e88fc99b254a-m0.avro".to_string(), + manifest_length: 6926, + partition_spec_id: 1, + content: ManifestContentType::Data, + sequence_number: 1, + min_sequence_number: 1, + added_snapshot_id: 377075049360453639, + added_data_files_count: Some(1), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::long(1)), upper_bound: Some(Literal::long(1))}], + key_metadata: vec![], + }] + }.try_into().unwrap(); + let result = serde_json::to_string(&manifest_list).unwrap(); + assert_eq!( + result, + r#"[{"manifest_path":"s3a://icebergdata/demo/s1/t1/metadata/05ffe08b-810f-49b3-a8f4-e88fc99b254a-m0.avro","manifest_length":6926,"partition_spec_id":1,"content":0,"sequence_number":1,"min_sequence_number":1,"added_snapshot_id":377075049360453639,"added_data_files_count":1,"existing_data_files_count":0,"deleted_data_files_count":0,"added_rows_count":3,"existing_rows_count":0,"deleted_rows_count":0,"partitions":[{"contains_null":false,"contains_nan":false,"lower_bound":[1,0,0,0,0,0,0,0],"upper_bound":[1,0,0,0,0,0,0,0]}],"key_metadata":null}]"# + ); + } + + #[tokio::test] + async fn test_manifest_list_writer_v1() { + let expected_manifest_list = ManifestList { + entries: vec![ManifestListEntry { + manifest_path: "/opt/bitnami/spark/warehouse/db/table/metadata/10d28031-9739-484c-92db-cdf2975cead4-m0.avro".to_string(), + manifest_length: 5806, + partition_spec_id: 1, + content: ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + added_snapshot_id: 1646658105718557341, + added_data_files_count: Some(3), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::long(1)), upper_bound: Some(Literal::long(1))}], + key_metadata: vec![], + }] + }; + + let temp_dir = TempDir::new().unwrap(); + let path = temp_dir.path().join("manifest_list_v1.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(path.to_str().unwrap()).unwrap(); + + let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, 0); + writer + .add_manifest_entries(expected_manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + + let bs = fs::read(path).unwrap(); + + let partition_types = HashMap::from([( + 1, + StructType::new(vec![Arc::new(NestedField::required( + 1, + "test", + Type::Primitive(PrimitiveType::Long), + ))]), + )]); + + let manifest_list = + ManifestList::parse_with_version(&bs, crate::spec::FormatVersion::V1, move |id| { + Ok(partition_types.get(&id).cloned()) + }) + .unwrap(); + assert_eq!(manifest_list, expected_manifest_list); + + temp_dir.close().unwrap(); + } + + #[tokio::test] + async fn test_manifest_list_writer_v2() { + let snapshot_id = 377075049360453639; + let seq_num = 1; + let mut expected_manifest_list = ManifestList { + entries: vec![ManifestListEntry { + manifest_path: "s3a://icebergdata/demo/s1/t1/metadata/05ffe08b-810f-49b3-a8f4-e88fc99b254a-m0.avro".to_string(), + manifest_length: 6926, + partition_spec_id: 1, + content: ManifestContentType::Data, + sequence_number: UNASSIGNED_SEQUENCE_NUMBER, + min_sequence_number: UNASSIGNED_SEQUENCE_NUMBER, + added_snapshot_id: snapshot_id, + added_data_files_count: Some(1), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::long(1)), upper_bound: Some(Literal::long(1))}], + key_metadata: vec![], + }] + }; + + let temp_dir = TempDir::new().unwrap(); + let path = temp_dir.path().join("manifest_list_v2.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(path.to_str().unwrap()).unwrap(); + + let mut writer = ManifestListWriter::v2(output_file, snapshot_id, 0, seq_num); + writer + .add_manifest_entries(expected_manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + + let bs = fs::read(path).unwrap(); + let partition_types = HashMap::from([( + 1, + StructType::new(vec![Arc::new(NestedField::required( + 1, + "test", + Type::Primitive(PrimitiveType::Long), + ))]), + )]); + let manifest_list = + ManifestList::parse_with_version(&bs, crate::spec::FormatVersion::V2, move |id| { + Ok(partition_types.get(&id).cloned()) + }) + .unwrap(); + expected_manifest_list.entries[0].sequence_number = seq_num; + expected_manifest_list.entries[0].min_sequence_number = seq_num; + assert_eq!(manifest_list, expected_manifest_list); + + temp_dir.close().unwrap(); + } + + #[tokio::test] + async fn test_manifest_list_writer_v1_as_v2() { + let expected_manifest_list = ManifestList { + entries: vec![ManifestListEntry { + manifest_path: "/opt/bitnami/spark/warehouse/db/table/metadata/10d28031-9739-484c-92db-cdf2975cead4-m0.avro".to_string(), + manifest_length: 5806, + partition_spec_id: 1, + content: ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + added_snapshot_id: 1646658105718557341, + added_data_files_count: Some(3), + existing_data_files_count: Some(0), + deleted_data_files_count: Some(0), + added_rows_count: Some(3), + existing_rows_count: Some(0), + deleted_rows_count: Some(0), + partitions: vec![FieldSummary { contains_null: false, contains_nan: Some(false), lower_bound: Some(Literal::long(1)), upper_bound: Some(Literal::long(1))}], + key_metadata: vec![], + }] + }; + + let temp_dir = TempDir::new().unwrap(); + let path = temp_dir.path().join("manifest_list_v1.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(path.to_str().unwrap()).unwrap(); + + let mut writer = ManifestListWriter::v2(output_file, 1646658105718557341, 0, 1); + writer + .add_manifest_entries(expected_manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + + let bs = fs::read(path).unwrap(); + + let partition_types = HashMap::from([( + 1, + StructType::new(vec![Arc::new(NestedField::required( + 1, + "test", + Type::Primitive(PrimitiveType::Long), + ))]), + )]); + + let manifest_list = + ManifestList::parse_with_version(&bs, crate::spec::FormatVersion::V2, move |id| { + Ok(partition_types.get(&id).cloned()) + }) + .unwrap(); + assert_eq!(manifest_list, expected_manifest_list); + + temp_dir.close().unwrap(); + } +} diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs new file mode 100644 index 0000000..199fc4a --- /dev/null +++ b/crates/iceberg/src/spec/mod.rs @@ -0,0 +1,40 @@ +// 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. + +//! Spec for Iceberg. + +mod datatypes; +mod manifest; +mod manifest_list; +mod partition; +mod schema; +mod snapshot; +mod sort; +mod table_metadata; +mod transform; +mod values; + +pub use datatypes::*; +pub use manifest::*; +pub use manifest_list::*; +pub use partition::*; +pub use schema::*; +pub use snapshot::*; +pub use sort::*; +pub use table_metadata::*; +pub use transform::*; +pub use values::*; diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs new file mode 100644 index 0000000..9388820 --- /dev/null +++ b/crates/iceberg/src/spec/partition.rs @@ -0,0 +1,492 @@ +// 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. + +/*! + * Partitioning +*/ +use serde::{Deserialize, Serialize}; +use std::sync::Arc; +use typed_builder::TypedBuilder; + +use crate::{Error, ErrorKind}; + +use super::{transform::Transform, NestedField, Schema, StructType}; + +/// Reference to [`PartitionSpec`]. +pub type PartitionSpecRef = Arc; +/// Partition fields capture the transform from table data to partition values. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, TypedBuilder)] +#[serde(rename_all = "kebab-case")] +pub struct PartitionField { + /// A source column id from the table’s schema + pub source_id: i32, + /// A partition field id that is used to identify a partition field and is unique within a partition spec. + /// In v2 table metadata, it is unique across all partition specs. + pub field_id: i32, + /// A partition name. + pub name: String, + /// A transform that is applied to the source column to produce a partition value. + pub transform: Transform, +} + +/// Partition spec that defines how to produce a tuple of partition values from a record. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Default, Builder)] +#[serde(rename_all = "kebab-case")] +#[builder(setter(prefix = "with"))] +pub struct PartitionSpec { + /// Identifier for PartitionSpec + pub spec_id: i32, + /// Details of the partition spec + #[builder(setter(each(name = "with_partition_field")))] + pub fields: Vec, +} + +impl PartitionSpec { + /// Create partition spec builer + pub fn builder() -> PartitionSpecBuilder { + PartitionSpecBuilder::default() + } + + /// Returns if the partition spec is unpartitioned. + /// + /// A [`PartitionSpec`] is unpartitioned if it has no fields or all fields are [`Transform::Void`] transform. + pub fn is_unpartitioned(&self) -> bool { + self.fields.is_empty() + || self + .fields + .iter() + .all(|f| matches!(f.transform, Transform::Void)) + } + + /// Returns the partition type of this partition spec. + pub fn partition_type(&self, schema: &Schema) -> Result { + let mut fields = Vec::with_capacity(self.fields.len()); + for partition_field in &self.fields { + let field = schema + .field_by_id(partition_field.source_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "No column with source column id {} in schema {:?}", + partition_field.source_id, schema + ), + ) + })?; + let res_type = partition_field.transform.result_type(&field.field_type)?; + let field = + NestedField::optional(partition_field.field_id, &partition_field.name, res_type) + .into(); + fields.push(field); + } + Ok(StructType::new(fields)) + } +} + +/// Reference to [`UnboundPartitionSpec`]. +pub type UnboundPartitionSpecRef = Arc; +/// Unbound partition field can be built without a schema and later bound to a schema. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, TypedBuilder)] +#[serde(rename_all = "kebab-case")] +pub struct UnboundPartitionField { + /// A source column id from the table’s schema + pub source_id: i32, + /// A partition field id that is used to identify a partition field and is unique within a partition spec. + /// In v2 table metadata, it is unique across all partition specs. + #[builder(default, setter(strip_option))] + pub partition_id: Option, + /// A partition name. + pub name: String, + /// A transform that is applied to the source column to produce a partition value. + pub transform: Transform, +} + +/// Unbound partition spec can be built without a schema and later bound to a schema. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Default, Builder)] +#[serde(rename_all = "kebab-case")] +#[builder(setter(prefix = "with"))] +pub struct UnboundPartitionSpec { + /// Identifier for PartitionSpec + #[builder(default, setter(strip_option))] + pub spec_id: Option, + /// Details of the partition spec + #[builder(setter(each(name = "with_unbound_partition_field")))] + pub fields: Vec, +} + +impl UnboundPartitionSpec { + /// Create unbound partition spec builer + pub fn builder() -> UnboundPartitionSpecBuilder { + UnboundPartitionSpecBuilder::default() + } +} + +#[cfg(test)] +mod tests { + use crate::spec::Type; + + use super::*; + + #[test] + fn test_partition_spec() { + let spec = r#" + { + "spec-id": 1, + "fields": [ { + "source-id": 4, + "field-id": 1000, + "name": "ts_day", + "transform": "day" + }, { + "source-id": 1, + "field-id": 1001, + "name": "id_bucket", + "transform": "bucket[16]" + }, { + "source-id": 2, + "field-id": 1002, + "name": "id_truncate", + "transform": "truncate[4]" + } ] + } + "#; + + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + assert_eq!(4, partition_spec.fields[0].source_id); + assert_eq!(1000, partition_spec.fields[0].field_id); + assert_eq!("ts_day", partition_spec.fields[0].name); + assert_eq!(Transform::Day, partition_spec.fields[0].transform); + + assert_eq!(1, partition_spec.fields[1].source_id); + assert_eq!(1001, partition_spec.fields[1].field_id); + assert_eq!("id_bucket", partition_spec.fields[1].name); + assert_eq!(Transform::Bucket(16), partition_spec.fields[1].transform); + + assert_eq!(2, partition_spec.fields[2].source_id); + assert_eq!(1002, partition_spec.fields[2].field_id); + assert_eq!("id_truncate", partition_spec.fields[2].name); + assert_eq!(Transform::Truncate(4), partition_spec.fields[2].transform); + } + + #[test] + fn test_is_unpartitioned() { + let partition_spec = PartitionSpec::builder() + .with_spec_id(1) + .with_fields(vec![]) + .build() + .unwrap(); + assert!( + partition_spec.is_unpartitioned(), + "Empty partition spec should be unpartitioned" + ); + + let partition_spec = PartitionSpec::builder() + .with_partition_field( + PartitionField::builder() + .source_id(1) + .field_id(1) + .name("id".to_string()) + .transform(Transform::Identity) + .build(), + ) + .with_partition_field( + PartitionField::builder() + .source_id(2) + .field_id(2) + .name("name".to_string()) + .transform(Transform::Void) + .build(), + ) + .with_spec_id(1) + .build() + .unwrap(); + assert!( + !partition_spec.is_unpartitioned(), + "Partition spec with one non void transform should not be unpartitioned" + ); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(1) + .with_partition_field( + PartitionField::builder() + .source_id(1) + .field_id(1) + .name("id".to_string()) + .transform(Transform::Void) + .build(), + ) + .with_partition_field( + PartitionField::builder() + .source_id(2) + .field_id(2) + .name("name".to_string()) + .transform(Transform::Void) + .build(), + ) + .build() + .unwrap(); + assert!( + partition_spec.is_unpartitioned(), + "Partition spec with all void field should be unpartitioned" + ); + } + + #[test] + fn test_unbound_partition_spec() { + let spec = r#" + { + "spec-id": 1, + "fields": [ { + "source-id": 4, + "partition-id": 1000, + "name": "ts_day", + "transform": "day" + }, { + "source-id": 1, + "partition-id": 1001, + "name": "id_bucket", + "transform": "bucket[16]" + }, { + "source-id": 2, + "partition-id": 1002, + "name": "id_truncate", + "transform": "truncate[4]" + } ] + } + "#; + + let partition_spec: UnboundPartitionSpec = serde_json::from_str(spec).unwrap(); + assert_eq!(Some(1), partition_spec.spec_id); + + assert_eq!(4, partition_spec.fields[0].source_id); + assert_eq!(Some(1000), partition_spec.fields[0].partition_id); + assert_eq!("ts_day", partition_spec.fields[0].name); + assert_eq!(Transform::Day, partition_spec.fields[0].transform); + + assert_eq!(1, partition_spec.fields[1].source_id); + assert_eq!(Some(1001), partition_spec.fields[1].partition_id); + assert_eq!("id_bucket", partition_spec.fields[1].name); + assert_eq!(Transform::Bucket(16), partition_spec.fields[1].transform); + + assert_eq!(2, partition_spec.fields[2].source_id); + assert_eq!(Some(1002), partition_spec.fields[2].partition_id); + assert_eq!("id_truncate", partition_spec.fields[2].name); + assert_eq!(Transform::Truncate(4), partition_spec.fields[2].transform); + + let spec = r#" + { + "fields": [ { + "source-id": 4, + "name": "ts_day", + "transform": "day" + } ] + } + "#; + let partition_spec: UnboundPartitionSpec = serde_json::from_str(spec).unwrap(); + assert_eq!(None, partition_spec.spec_id); + + assert_eq!(4, partition_spec.fields[0].source_id); + assert_eq!(None, partition_spec.fields[0].partition_id); + assert_eq!("ts_day", partition_spec.fields[0].name); + assert_eq!(Transform::Day, partition_spec.fields[0].transform); + } + + #[test] + fn test_partition_type() { + let spec = r#" + { + "spec-id": 1, + "fields": [ { + "source-id": 4, + "field-id": 1000, + "name": "ts_day", + "transform": "day" + }, { + "source-id": 1, + "field-id": 1001, + "name": "id_bucket", + "transform": "bucket[16]" + }, { + "source-id": 2, + "field-id": 1002, + "name": "id_truncate", + "transform": "truncate[4]" + } ] + } + "#; + + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let schema = Schema::builder() + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) + .into(), + NestedField::required( + 2, + "name", + Type::Primitive(crate::spec::PrimitiveType::String), + ) + .into(), + NestedField::required( + 3, + "ts", + Type::Primitive(crate::spec::PrimitiveType::Timestamp), + ) + .into(), + NestedField::required( + 4, + "ts_day", + Type::Primitive(crate::spec::PrimitiveType::Timestamp), + ) + .into(), + NestedField::required( + 5, + "id_bucket", + Type::Primitive(crate::spec::PrimitiveType::Int), + ) + .into(), + NestedField::required( + 6, + "id_truncate", + Type::Primitive(crate::spec::PrimitiveType::Int), + ) + .into(), + ]) + .build() + .unwrap(); + + let partition_type = partition_spec.partition_type(&schema).unwrap(); + assert_eq!(3, partition_type.fields().len()); + assert_eq!( + *partition_type.fields()[0], + NestedField::optional( + partition_spec.fields[0].field_id, + &partition_spec.fields[0].name, + Type::Primitive(crate::spec::PrimitiveType::Int) + ) + ); + assert_eq!( + *partition_type.fields()[1], + NestedField::optional( + partition_spec.fields[1].field_id, + &partition_spec.fields[1].name, + Type::Primitive(crate::spec::PrimitiveType::Int) + ) + ); + assert_eq!( + *partition_type.fields()[2], + NestedField::optional( + partition_spec.fields[2].field_id, + &partition_spec.fields[2].name, + Type::Primitive(crate::spec::PrimitiveType::String) + ) + ); + } + + #[test] + fn test_partition_empty() { + let spec = r#" + { + "spec-id": 1, + "fields": [] + } + "#; + + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let schema = Schema::builder() + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) + .into(), + NestedField::required( + 2, + "name", + Type::Primitive(crate::spec::PrimitiveType::String), + ) + .into(), + NestedField::required( + 3, + "ts", + Type::Primitive(crate::spec::PrimitiveType::Timestamp), + ) + .into(), + NestedField::required( + 4, + "ts_day", + Type::Primitive(crate::spec::PrimitiveType::Timestamp), + ) + .into(), + NestedField::required( + 5, + "id_bucket", + Type::Primitive(crate::spec::PrimitiveType::Int), + ) + .into(), + NestedField::required( + 6, + "id_truncate", + Type::Primitive(crate::spec::PrimitiveType::Int), + ) + .into(), + ]) + .build() + .unwrap(); + + let partition_type = partition_spec.partition_type(&schema).unwrap(); + assert_eq!(0, partition_type.fields().len()); + } + + #[test] + fn test_partition_error() { + let spec = r#" + { + "spec-id": 1, + "fields": [ { + "source-id": 4, + "field-id": 1000, + "name": "ts_day", + "transform": "day" + }, { + "source-id": 1, + "field-id": 1001, + "name": "id_bucket", + "transform": "bucket[16]" + }, { + "source-id": 2, + "field-id": 1002, + "name": "id_truncate", + "transform": "truncate[4]" + } ] + } + "#; + + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let schema = Schema::builder() + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) + .into(), + NestedField::required( + 2, + "name", + Type::Primitive(crate::spec::PrimitiveType::String), + ) + .into(), + ]) + .build() + .unwrap(); + + assert!(partition_spec.partition_type(&schema).is_err()); + } +} diff --git a/crates/iceberg/src/spec/schema.rs b/crates/iceberg/src/spec/schema.rs new file mode 100644 index 0000000..34e383f --- /dev/null +++ b/crates/iceberg/src/spec/schema.rs @@ -0,0 +1,1289 @@ +// 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. + +//! This module defines schema in iceberg. + +use crate::error::Result; +use crate::spec::datatypes::{ + ListType, MapType, NestedFieldRef, PrimitiveType, StructType, Type, LIST_FILED_NAME, + MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME, +}; +use crate::{ensure_data_valid, Error, ErrorKind}; +use bimap::BiHashMap; +use itertools::Itertools; +use serde::{Deserialize, Serialize}; +use std::collections::{HashMap, HashSet}; +use std::fmt::{Display, Formatter}; +use std::sync::Arc; + +use _serde::SchemaEnum; + +/// Type alias for schema id. +pub type SchemaId = i32; +/// Reference to [`Schema`]. +pub type SchemaRef = Arc; +const DEFAULT_SCHEMA_ID: SchemaId = 0; + +/// Defines schema in iceberg. +#[derive(Debug, Serialize, Deserialize, Clone)] +#[serde(try_from = "SchemaEnum", into = "SchemaEnum")] +pub struct Schema { + r#struct: StructType, + schema_id: SchemaId, + highest_field_id: i32, + identifier_field_ids: HashSet, + + alias_to_id: BiHashMap, + id_to_field: HashMap, + + name_to_id: HashMap, + id_to_name: HashMap, +} + +impl PartialEq for Schema { + fn eq(&self, other: &Self) -> bool { + self.r#struct == other.r#struct + && self.schema_id == other.schema_id + && self.identifier_field_ids == other.identifier_field_ids + } +} + +impl Eq for Schema {} + +/// Schema builder. +#[derive(Debug)] +pub struct SchemaBuilder { + schema_id: i32, + fields: Vec, + alias_to_id: BiHashMap, + identifier_field_ids: HashSet, +} + +impl SchemaBuilder { + /// Add fields to schema builder. + pub fn with_fields(mut self, fields: impl IntoIterator) -> Self { + self.fields.extend(fields); + self + } + + /// Set schema id. + pub fn with_schema_id(mut self, schema_id: i32) -> Self { + self.schema_id = schema_id; + self + } + + /// Set identifier field ids. + pub fn with_identifier_field_ids(mut self, ids: impl IntoIterator) -> Self { + self.identifier_field_ids.extend(ids); + self + } + + /// Set alias to filed id mapping. + pub fn with_alias(mut self, alias_to_id: BiHashMap) -> Self { + self.alias_to_id = alias_to_id; + self + } + + /// Builds the schema. + pub fn build(self) -> Result { + let highest_field_id = self.fields.iter().map(|f| f.id).max().unwrap_or(0); + + let r#struct = StructType::new(self.fields); + let id_to_field = index_by_id(&r#struct)?; + + Self::validate_identifier_ids( + &r#struct, + &id_to_field, + self.identifier_field_ids.iter().copied(), + )?; + + let (name_to_id, id_to_name) = { + let mut index = IndexByName::default(); + visit_struct(&r#struct, &mut index)?; + index.indexes() + }; + + Ok(Schema { + r#struct, + schema_id: self.schema_id, + highest_field_id, + identifier_field_ids: self.identifier_field_ids, + + alias_to_id: self.alias_to_id, + id_to_field, + + name_to_id, + id_to_name, + }) + } + + fn validate_identifier_ids( + r#struct: &StructType, + id_to_field: &HashMap, + identifier_field_ids: impl Iterator, + ) -> Result<()> { + let id_to_parent = index_parents(r#struct)?; + for identifier_field_id in identifier_field_ids { + let field = id_to_field.get(&identifier_field_id).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot add identifier field {identifier_field_id}: field does not exist" + ), + ) + })?; + ensure_data_valid!( + field.required, + "Cannot add identifier field: {} is an optional field", + field.name + ); + if let Type::Primitive(p) = field.field_type.as_ref() { + ensure_data_valid!( + !matches!(p, PrimitiveType::Double | PrimitiveType::Float), + "Cannot add identifier field {}: cannot be a float or double type", + field.name + ); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot add field {} as an identifier field: not a primitive type field", + field.name + ), + )); + } + + let mut cur_field_id = identifier_field_id; + while let Some(parent) = id_to_parent.get(&cur_field_id) { + let parent_field = id_to_field + .get(parent) + .expect("Field id should not disappear."); + ensure_data_valid!( + parent_field.field_type.is_struct(), + "Cannot add field {} as an identifier field: must not be nested in {:?}", + field.name, + parent_field + ); + ensure_data_valid!(parent_field.required, "Cannot add field {} as an identifier field: must not be nested in an optional field {}", field.name, parent_field); + cur_field_id = *parent; + } + } + + Ok(()) + } +} + +impl Schema { + /// Create a schema builder. + pub fn builder() -> SchemaBuilder { + SchemaBuilder { + schema_id: DEFAULT_SCHEMA_ID, + fields: vec![], + identifier_field_ids: HashSet::default(), + alias_to_id: BiHashMap::default(), + } + } + + /// Get field by field id. + pub fn field_by_id(&self, field_id: i32) -> Option<&NestedFieldRef> { + self.id_to_field.get(&field_id) + } + + /// Get field by field name. + /// + /// Both full name and short name could work here. + pub fn field_by_name(&self, field_name: &str) -> Option<&NestedFieldRef> { + self.name_to_id + .get(field_name) + .and_then(|id| self.field_by_id(*id)) + } + + /// Get field by alias. + pub fn field_by_alias(&self, alias: &str) -> Option<&NestedFieldRef> { + self.alias_to_id + .get_by_left(alias) + .and_then(|id| self.field_by_id(*id)) + } + + /// Returns [`highest_field_id`]. + #[inline] + pub fn highest_field_id(&self) -> i32 { + self.highest_field_id + } + + /// Returns [`schema_id`]. + #[inline] + pub fn schema_id(&self) -> i32 { + self.schema_id + } + + /// Returns [`r#struct`]. + #[inline] + pub fn as_struct(&self) -> &StructType { + &self.r#struct + } + + /// Get field id by full name. + pub fn field_id_by_name(&self, name: &str) -> Option { + self.name_to_id.get(name).copied() + } + + /// Get field id by full name. + pub fn name_by_field_id(&self, field_id: i32) -> Option<&str> { + self.id_to_name.get(&field_id).map(String::as_str) + } +} + +impl Display for Schema { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "table {{")?; + for field in self.as_struct().fields() { + writeln!(f, " {}", field)?; + } + writeln!(f, "}}") + } +} + +/// A post order schema visitor. +/// +/// For order of methods called, please refer to [`visit_schema`]. +pub trait SchemaVisitor { + /// Return type of this visitor. + type T; + + /// Called before struct field. + fn before_struct_field(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called after struct field. + fn after_struct_field(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called before list field. + fn before_list_element(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called after list field. + fn after_list_element(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called before map key field. + fn before_map_key(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called after map key field. + fn after_map_key(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called before map value field. + fn before_map_value(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + /// Called after map value field. + fn after_map_value(&mut self, _field: &NestedFieldRef) -> Result<()> { + Ok(()) + } + + /// Called after schema's type visited. + fn schema(&mut self, schema: &Schema, value: Self::T) -> Result; + /// Called after struct's field type visited. + fn field(&mut self, field: &NestedFieldRef, value: Self::T) -> Result; + /// Called after struct's fields visited. + fn r#struct(&mut self, r#struct: &StructType, results: Vec) -> Result; + /// Called after list fields visited. + fn list(&mut self, list: &ListType, value: Self::T) -> Result; + /// Called after map's key and value fields visited. + fn map(&mut self, map: &MapType, key_value: Self::T, value: Self::T) -> Result; + /// Called when see a primitive type. + fn primitive(&mut self, p: &PrimitiveType) -> Result; +} + +/// Visiting a type in post order. +pub fn visit_type(r#type: &Type, visitor: &mut V) -> Result { + match r#type { + Type::Primitive(p) => visitor.primitive(p), + Type::List(list) => { + visitor.before_list_element(&list.element_field)?; + let value = visit_type(&list.element_field.field_type, visitor)?; + visitor.after_list_element(&list.element_field)?; + visitor.list(list, value) + } + Type::Map(map) => { + let key_result = { + visitor.before_map_key(&map.key_field)?; + let ret = visit_type(&map.key_field.field_type, visitor)?; + visitor.after_map_key(&map.key_field)?; + ret + }; + + let value_result = { + visitor.before_map_value(&map.value_field)?; + let ret = visit_type(&map.value_field.field_type, visitor)?; + visitor.after_map_value(&map.value_field)?; + ret + }; + + visitor.map(map, key_result, value_result) + } + Type::Struct(s) => visit_struct(s, visitor), + } +} + +/// Visit struct type in post order. +pub fn visit_struct(s: &StructType, visitor: &mut V) -> Result { + let mut results = Vec::with_capacity(s.fields().len()); + for field in s.fields() { + visitor.before_struct_field(field)?; + let result = visit_type(&field.field_type, visitor)?; + visitor.after_struct_field(field)?; + let result = visitor.field(field, result)?; + results.push(result); + } + + visitor.r#struct(s, results) +} + +/// Visit schema in post order. +pub fn visit_schema(schema: &Schema, visitor: &mut V) -> Result { + let result = visit_struct(&schema.r#struct, visitor)?; + visitor.schema(schema, result) +} + +/// Creates an field id to field map. +pub fn index_by_id(r#struct: &StructType) -> Result> { + struct IndexById(HashMap); + + impl SchemaVisitor for IndexById { + type T = (); + + fn schema(&mut self, _schema: &Schema, _value: ()) -> Result<()> { + Ok(()) + } + + fn field(&mut self, field: &NestedFieldRef, _value: ()) -> Result<()> { + self.0.insert(field.id, field.clone()); + Ok(()) + } + + fn r#struct(&mut self, _struct: &StructType, _results: Vec) -> Result { + Ok(()) + } + + fn list(&mut self, list: &ListType, _value: Self::T) -> Result { + self.0 + .insert(list.element_field.id, list.element_field.clone()); + Ok(()) + } + + fn map(&mut self, map: &MapType, _key_value: Self::T, _value: Self::T) -> Result { + self.0.insert(map.key_field.id, map.key_field.clone()); + self.0.insert(map.value_field.id, map.value_field.clone()); + Ok(()) + } + + fn primitive(&mut self, _: &PrimitiveType) -> Result { + Ok(()) + } + } + + let mut index = IndexById(HashMap::new()); + visit_struct(r#struct, &mut index)?; + Ok(index.0) +} + +/// Creates a field id to parent field id map. +pub fn index_parents(r#struct: &StructType) -> Result> { + struct IndexByParent { + parents: Vec, + result: HashMap, + } + + impl SchemaVisitor for IndexByParent { + type T = (); + + fn before_struct_field(&mut self, field: &NestedFieldRef) -> Result<()> { + self.parents.push(field.id); + Ok(()) + } + + fn after_struct_field(&mut self, _field: &NestedFieldRef) -> Result<()> { + self.parents.pop(); + Ok(()) + } + + fn before_list_element(&mut self, field: &NestedFieldRef) -> Result<()> { + self.parents.push(field.id); + Ok(()) + } + + fn after_list_element(&mut self, _field: &NestedFieldRef) -> Result<()> { + self.parents.pop(); + Ok(()) + } + + fn before_map_key(&mut self, field: &NestedFieldRef) -> Result<()> { + self.parents.push(field.id); + Ok(()) + } + + fn after_map_key(&mut self, _field: &NestedFieldRef) -> Result<()> { + self.parents.pop(); + Ok(()) + } + + fn before_map_value(&mut self, field: &NestedFieldRef) -> Result<()> { + self.parents.push(field.id); + Ok(()) + } + + fn after_map_value(&mut self, _field: &NestedFieldRef) -> Result<()> { + self.parents.pop(); + Ok(()) + } + + fn schema(&mut self, _schema: &Schema, _value: Self::T) -> Result { + Ok(()) + } + + fn field(&mut self, field: &NestedFieldRef, _value: Self::T) -> Result { + if let Some(parent) = self.parents.last().copied() { + self.result.insert(field.id, parent); + } + Ok(()) + } + + fn r#struct(&mut self, _struct: &StructType, _results: Vec) -> Result { + Ok(()) + } + + fn list(&mut self, _list: &ListType, _value: Self::T) -> Result { + Ok(()) + } + + fn map(&mut self, _map: &MapType, _key_value: Self::T, _value: Self::T) -> Result { + Ok(()) + } + + fn primitive(&mut self, _p: &PrimitiveType) -> Result { + Ok(()) + } + } + + let mut index = IndexByParent { + parents: vec![], + result: HashMap::new(), + }; + visit_struct(r#struct, &mut index)?; + Ok(index.result) +} + +#[derive(Default)] +struct IndexByName { + // Maybe radix tree is better here? + name_to_id: HashMap, + short_name_to_id: HashMap, + + field_names: Vec, + short_field_names: Vec, +} + +impl IndexByName { + fn add_field(&mut self, name: &str, field_id: i32) -> Result<()> { + let full_name = self + .field_names + .iter() + .map(String::as_str) + .chain(vec![name]) + .join("."); + if let Some(existing_field_id) = self.name_to_id.get(full_name.as_str()) { + return Err(Error::new(ErrorKind::DataInvalid, format!("Invalid schema: multiple fields for name {full_name}: {field_id} and {existing_field_id}"))); + } else { + self.name_to_id.insert(full_name, field_id); + } + + let full_short_name = self + .short_field_names + .iter() + .map(String::as_str) + .chain(vec![name]) + .join("."); + self.short_name_to_id + .entry(full_short_name) + .or_insert_with(|| field_id); + Ok(()) + } + + /// Returns two indexes: full name to field id, and id to full name. + /// + /// In the first index, short names are returned. + /// In second index, short names are not returned. + pub fn indexes(mut self) -> (HashMap, HashMap) { + self.short_name_to_id.reserve(self.name_to_id.len()); + for (name, id) in &self.name_to_id { + self.short_name_to_id.insert(name.clone(), *id); + } + + let id_to_name = self.name_to_id.into_iter().map(|e| (e.1, e.0)).collect(); + (self.short_name_to_id, id_to_name) + } +} + +impl SchemaVisitor for IndexByName { + type T = (); + + fn before_struct_field(&mut self, field: &NestedFieldRef) -> Result<()> { + self.field_names.push(field.name.to_string()); + self.short_field_names.push(field.name.to_string()); + Ok(()) + } + + fn after_struct_field(&mut self, _field: &NestedFieldRef) -> Result<()> { + self.field_names.pop(); + self.short_field_names.pop(); + Ok(()) + } + + fn before_list_element(&mut self, field: &NestedFieldRef) -> Result<()> { + self.field_names.push(field.name.clone()); + if !field.field_type.is_struct() { + self.short_field_names.push(field.name.to_string()); + } + + Ok(()) + } + + fn after_list_element(&mut self, field: &NestedFieldRef) -> Result<()> { + self.field_names.pop(); + if !field.field_type.is_struct() { + self.short_field_names.pop(); + } + + Ok(()) + } + + fn before_map_key(&mut self, field: &NestedFieldRef) -> Result<()> { + self.before_struct_field(field) + } + + fn after_map_key(&mut self, field: &NestedFieldRef) -> Result<()> { + self.after_struct_field(field) + } + + fn before_map_value(&mut self, field: &NestedFieldRef) -> Result<()> { + self.field_names.push(field.name.to_string()); + if !field.field_type.is_struct() { + self.short_field_names.push(field.name.to_string()); + } + Ok(()) + } + + fn after_map_value(&mut self, field: &NestedFieldRef) -> Result<()> { + self.field_names.pop(); + if !field.field_type.is_struct() { + self.short_field_names.pop(); + } + + Ok(()) + } + + fn schema(&mut self, _schema: &Schema, _value: Self::T) -> Result { + Ok(()) + } + + fn field(&mut self, field: &NestedFieldRef, _value: Self::T) -> Result { + self.add_field(field.name.as_str(), field.id) + } + + fn r#struct(&mut self, _struct: &StructType, _results: Vec) -> Result { + Ok(()) + } + + fn list(&mut self, list: &ListType, _value: Self::T) -> Result { + self.add_field(LIST_FILED_NAME, list.element_field.id) + } + + fn map(&mut self, map: &MapType, _key_value: Self::T, _value: Self::T) -> Result { + self.add_field(MAP_KEY_FIELD_NAME, map.key_field.id)?; + self.add_field(MAP_VALUE_FIELD_NAME, map.value_field.id) + } + + fn primitive(&mut self, _p: &PrimitiveType) -> Result { + Ok(()) + } +} + +pub(super) mod _serde { + /// This is a helper module that defines types to help with serialization/deserialization. + /// For deserialization the input first gets read into either the [SchemaV1] or [SchemaV2] struct + /// and then converted into the [Schema] struct. Serialization works the other way around. + /// [SchemaV1] and [SchemaV2] are internal struct that are only used for serialization and deserialization. + use serde::{Deserialize, Serialize}; + + use crate::{spec::StructType, Error, Result}; + + use super::{Schema, DEFAULT_SCHEMA_ID}; + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(untagged)] + /// Enum for Schema serialization/deserializaion + pub(super) enum SchemaEnum { + V2(SchemaV2), + V1(SchemaV1), + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v2 schema for serialization/deserialization + pub(crate) struct SchemaV2 { + pub schema_id: i32, + #[serde(skip_serializing_if = "Option::is_none")] + pub identifier_field_ids: Option>, + #[serde(flatten)] + pub fields: StructType, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v1 schema for serialization/deserialization + pub(crate) struct SchemaV1 { + #[serde(skip_serializing_if = "Option::is_none")] + pub schema_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub identifier_field_ids: Option>, + #[serde(flatten)] + pub fields: StructType, + } + + /// Helper to serialize/deserializa Schema + impl TryFrom for Schema { + type Error = Error; + fn try_from(value: SchemaEnum) -> Result { + match value { + SchemaEnum::V2(value) => value.try_into(), + SchemaEnum::V1(value) => value.try_into(), + } + } + } + + impl From for SchemaEnum { + fn from(value: Schema) -> Self { + SchemaEnum::V2(value.into()) + } + } + + impl TryFrom for Schema { + type Error = Error; + fn try_from(value: SchemaV2) -> Result { + Schema::builder() + .with_schema_id(value.schema_id) + .with_fields(value.fields.fields().iter().cloned()) + .with_identifier_field_ids(value.identifier_field_ids.unwrap_or_default()) + .build() + } + } + + impl TryFrom for Schema { + type Error = Error; + fn try_from(value: SchemaV1) -> Result { + Schema::builder() + .with_schema_id(value.schema_id.unwrap_or(DEFAULT_SCHEMA_ID)) + .with_fields(value.fields.fields().iter().cloned()) + .with_identifier_field_ids(value.identifier_field_ids.unwrap_or_default()) + .build() + } + } + + impl From for SchemaV2 { + fn from(value: Schema) -> Self { + SchemaV2 { + schema_id: value.schema_id, + identifier_field_ids: if value.identifier_field_ids.is_empty() { + None + } else { + Some(value.identifier_field_ids.into_iter().collect()) + }, + fields: value.r#struct, + } + } + } + + impl From for SchemaV1 { + fn from(value: Schema) -> Self { + SchemaV1 { + schema_id: Some(value.schema_id), + identifier_field_ids: if value.identifier_field_ids.is_empty() { + None + } else { + Some(value.identifier_field_ids.into_iter().collect()) + }, + fields: value.r#struct, + } + } + } +} + +#[cfg(test)] +mod tests { + use crate::spec::datatypes::Type::{List, Map, Primitive, Struct}; + use crate::spec::datatypes::{ + ListType, MapType, NestedField, NestedFieldRef, PrimitiveType, StructType, Type, + }; + use crate::spec::schema::Schema; + use crate::spec::schema::_serde::{SchemaEnum, SchemaV1, SchemaV2}; + use std::collections::HashMap; + + use super::DEFAULT_SCHEMA_ID; + + fn check_schema_serde(json: &str, expected_type: Schema, _expected_enum: SchemaEnum) { + let desered_type: Schema = serde_json::from_str(json).unwrap(); + assert_eq!(desered_type, expected_type); + assert!(matches!(desered_type.clone(), _expected_enum)); + + let sered_json = serde_json::to_string(&expected_type).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + + assert_eq!(parsed_json_value, desered_type); + } + + #[test] + fn test_serde_with_schema_id() { + let (schema, record) = table_schema_simple(); + + let x: SchemaV2 = serde_json::from_str(record).unwrap(); + check_schema_serde(record, schema, SchemaEnum::V2(x)); + } + + #[test] + fn test_serde_without_schema_id() { + let (mut schema, record) = table_schema_simple(); + // we remove the ""schema-id": 1," string from example + let new_record = record.replace("\"schema-id\":1,", ""); + // By default schema_id field is set to DEFAULT_SCHEMA_ID when no value is set in json + schema.schema_id = DEFAULT_SCHEMA_ID; + + let x: SchemaV1 = serde_json::from_str(new_record.as_str()).unwrap(); + check_schema_serde(&new_record, schema, SchemaEnum::V1(x)); + } + + #[test] + fn test_construct_schema() { + let field1: NestedFieldRef = + NestedField::required(1, "f1", Type::Primitive(PrimitiveType::Boolean)).into(); + let field2: NestedFieldRef = + NestedField::optional(2, "f2", Type::Primitive(PrimitiveType::Int)).into(); + + let schema = Schema::builder() + .with_fields(vec![field1.clone()]) + .with_fields(vec![field2.clone()]) + .with_schema_id(3) + .build() + .unwrap(); + + assert_eq!(3, schema.schema_id()); + assert_eq!(2, schema.highest_field_id()); + assert_eq!(Some(&field1), schema.field_by_id(1)); + assert_eq!(Some(&field2), schema.field_by_id(2)); + assert_eq!(None, schema.field_by_id(3)); + } + + #[test] + fn schema() { + let record = r#" + { + "type": "struct", + "schema-id": 1, + "fields": [ { + "id": 1, + "name": "id", + "required": true, + "type": "uuid" + }, { + "id": 2, + "name": "data", + "required": false, + "type": "int" + } ] + } + "#; + + let result: SchemaV2 = serde_json::from_str(record).unwrap(); + assert_eq!(1, result.schema_id); + assert_eq!( + Box::new(Type::Primitive(PrimitiveType::Uuid)), + result.fields[0].field_type + ); + assert_eq!(1, result.fields[0].id); + assert!(result.fields[0].required); + + assert_eq!( + Box::new(Type::Primitive(PrimitiveType::Int)), + result.fields[1].field_type + ); + assert_eq!(2, result.fields[1].id); + assert!(!result.fields[1].required); + } + + fn table_schema_simple<'a>() -> (Schema, &'a str) { + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .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(), + ]) + .build() + .unwrap(); + let record = r#"{ + "type":"struct", + "schema-id":1, + "fields":[ + { + "id":1, + "name":"foo", + "required":false, + "type":"string" + }, + { + "id":2, + "name":"bar", + "required":true, + "type":"int" + }, + { + "id":3, + "name":"baz", + "required":false, + "type":"boolean" + } + ], + "identifier-field-ids":[2] + }"#; + (schema, record) + } + + fn table_schema_nested() -> Schema { + Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .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::required( + 4, + "qux", + Type::List(ListType { + element_field: NestedField::list_element( + 5, + Type::Primitive(PrimitiveType::String), + true, + ) + .into(), + }), + ) + .into(), + NestedField::required( + 6, + "quux", + Type::Map(MapType { + key_field: NestedField::map_key_element( + 7, + Type::Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 8, + Type::Map(MapType { + key_field: NestedField::map_key_element( + 9, + Type::Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 10, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + true, + ) + .into(), + }), + ) + .into(), + NestedField::required( + 11, + "location", + Type::List(ListType { + element_field: NestedField::list_element( + 12, + Type::Struct(StructType::new(vec![ + NestedField::optional( + 13, + "latitude", + Type::Primitive(PrimitiveType::Float), + ) + .into(), + NestedField::optional( + 14, + "longitude", + Type::Primitive(PrimitiveType::Float), + ) + .into(), + ])), + true, + ) + .into(), + }), + ) + .into(), + NestedField::optional( + 15, + "person", + Type::Struct(StructType::new(vec![ + NestedField::optional(16, "name", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::required(17, "age", Type::Primitive(PrimitiveType::Int)) + .into(), + ])), + ) + .into(), + ]) + .build() + .unwrap() + } + + #[test] + fn test_schema_display() { + let expected_str = " +table { + 1: foo: optional string\x20 + 2: bar: required int\x20 + 3: baz: optional boolean\x20 +} +"; + + assert_eq!(expected_str, format!("\n{}", table_schema_simple().0)); + } + + #[test] + fn test_schema_build_failed_on_duplicate_names() { + let ret = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![1]) + .with_fields(vec![ + NestedField::required(1, "foo", Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Primitive(PrimitiveType::Boolean)).into(), + NestedField::optional(4, "baz", Primitive(PrimitiveType::Boolean)).into(), + ]) + .build(); + + assert!(ret + .unwrap_err() + .message() + .contains("Invalid schema: multiple fields for name baz")); + } + + #[test] + fn test_schema_index_by_name() { + let expected_name_to_id = HashMap::from( + [ + ("foo", 1), + ("bar", 2), + ("baz", 3), + ("qux", 4), + ("qux.element", 5), + ("quux", 6), + ("quux.key", 7), + ("quux.value", 8), + ("quux.value.key", 9), + ("quux.value.value", 10), + ("location", 11), + ("location.element", 12), + ("location.element.latitude", 13), + ("location.element.longitude", 14), + ("location.latitude", 13), + ("location.longitude", 14), + ("person", 15), + ("person.name", 16), + ("person.age", 17), + ] + .map(|e| (e.0.to_string(), e.1)), + ); + + let schema = table_schema_nested(); + assert_eq!(&expected_name_to_id, &schema.name_to_id); + } + + #[test] + fn test_schema_find_column_name() { + let expected_column_name = HashMap::from([ + (1, "foo"), + (2, "bar"), + (3, "baz"), + (4, "qux"), + (5, "qux.element"), + (6, "quux"), + (7, "quux.key"), + (8, "quux.value"), + (9, "quux.value.key"), + (10, "quux.value.value"), + (11, "location"), + (12, "location.element"), + (13, "location.element.latitude"), + (14, "location.element.longitude"), + ]); + + let schema = table_schema_nested(); + for (id, name) in expected_column_name { + assert_eq!( + Some(name), + schema.name_by_field_id(id), + "Column name for field id {} not match.", + id + ); + } + } + + #[test] + fn test_schema_find_column_name_not_found() { + let schema = table_schema_nested(); + + assert!(schema.name_by_field_id(99).is_none()); + } + + #[test] + fn test_schema_find_column_name_by_id_simple() { + let expected_id_to_name = HashMap::from([(1, "foo"), (2, "bar"), (3, "baz")]); + + let schema = table_schema_simple().0; + + for (id, name) in expected_id_to_name { + assert_eq!( + Some(name), + schema.name_by_field_id(id), + "Column name for field id {} not match.", + id + ); + } + } + + #[test] + fn test_schema_find_simple() { + let schema = table_schema_simple().0; + + assert_eq!( + Some(schema.r#struct.fields()[0].clone()), + schema.field_by_id(1).cloned() + ); + assert_eq!( + Some(schema.r#struct.fields()[1].clone()), + schema.field_by_id(2).cloned() + ); + assert_eq!( + Some(schema.r#struct.fields()[2].clone()), + schema.field_by_id(3).cloned() + ); + + assert!(schema.field_by_id(4).is_none()); + assert!(schema.field_by_name("non exist").is_none()); + } + + #[test] + fn test_schema_find_nested() { + let expected_id_to_field: HashMap = HashMap::from([ + ( + 1, + NestedField::optional(1, "foo", Primitive(PrimitiveType::String)), + ), + ( + 2, + NestedField::required(2, "bar", Primitive(PrimitiveType::Int)), + ), + ( + 3, + NestedField::optional(3, "baz", Primitive(PrimitiveType::Boolean)), + ), + ( + 4, + NestedField::required( + 4, + "qux", + Type::List(ListType { + element_field: NestedField::list_element( + 5, + Type::Primitive(PrimitiveType::String), + true, + ) + .into(), + }), + ), + ), + ( + 5, + NestedField::required(5, "element", Primitive(PrimitiveType::String)), + ), + ( + 6, + NestedField::required( + 6, + "quux", + Map(MapType { + key_field: NestedField::map_key_element( + 7, + Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 8, + Map(MapType { + key_field: NestedField::map_key_element( + 9, + Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 10, + Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + true, + ) + .into(), + }), + ), + ), + ( + 7, + NestedField::required(7, "key", Primitive(PrimitiveType::String)), + ), + ( + 8, + NestedField::required( + 8, + "value", + Map(MapType { + key_field: NestedField::map_key_element( + 9, + Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 10, + Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ), + ), + ( + 9, + NestedField::required(9, "key", Primitive(PrimitiveType::String)), + ), + ( + 10, + NestedField::required(10, "value", Primitive(PrimitiveType::Int)), + ), + ( + 11, + NestedField::required( + 11, + "location", + List(ListType { + element_field: NestedField::list_element( + 12, + Struct(StructType::new(vec![ + NestedField::optional( + 13, + "latitude", + Primitive(PrimitiveType::Float), + ) + .into(), + NestedField::optional( + 14, + "longitude", + Primitive(PrimitiveType::Float), + ) + .into(), + ])), + true, + ) + .into(), + }), + ), + ), + ( + 12, + NestedField::list_element( + 12, + Struct(StructType::new(vec![ + NestedField::optional(13, "latitude", Primitive(PrimitiveType::Float)) + .into(), + NestedField::optional(14, "longitude", Primitive(PrimitiveType::Float)) + .into(), + ])), + true, + ), + ), + ( + 13, + NestedField::optional(13, "latitude", Primitive(PrimitiveType::Float)), + ), + ( + 14, + NestedField::optional(14, "longitude", Primitive(PrimitiveType::Float)), + ), + ( + 15, + NestedField::optional( + 15, + "person", + Type::Struct(StructType::new(vec![ + NestedField::optional(16, "name", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::required(17, "age", Type::Primitive(PrimitiveType::Int)) + .into(), + ])), + ), + ), + ( + 16, + NestedField::optional(16, "name", Type::Primitive(PrimitiveType::String)), + ), + ( + 17, + NestedField::required(17, "age", Type::Primitive(PrimitiveType::Int)), + ), + ]); + + let schema = table_schema_nested(); + for (id, field) in expected_id_to_field { + assert_eq!( + Some(&field), + schema.field_by_id(id).map(|f| f.as_ref()), + "Field for {} not match.", + id + ); + } + } +} diff --git a/crates/iceberg/src/spec/snapshot.rs b/crates/iceberg/src/spec/snapshot.rs new file mode 100644 index 0000000..3b4558b --- /dev/null +++ b/crates/iceberg/src/spec/snapshot.rs @@ -0,0 +1,404 @@ +// 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. + +/*! + * Snapshots +*/ +use crate::error::Result; +use chrono::{DateTime, TimeZone, Utc}; +use futures::AsyncReadExt; +use serde::{Deserialize, Serialize}; +use std::collections::HashMap; +use std::sync::Arc; +use typed_builder::TypedBuilder; + +use super::table_metadata::SnapshotLog; +use crate::io::FileIO; +use crate::spec::{ManifestList, SchemaId, SchemaRef, StructType, TableMetadata}; +use crate::{Error, ErrorKind}; +use _serde::SnapshotV2; + +/// Reference to [`Snapshot`]. +pub type SnapshotRef = Arc; +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "lowercase")] +/// The operation field is used by some operations, like snapshot expiration, to skip processing certain snapshots. +pub enum Operation { + /// Only data files were added and no files were removed. + Append, + /// Data and delete files were added and removed without changing table data; + /// i.e., compaction, changing the data file format, or relocating data files. + Replace, + /// Data and delete files were added and removed in a logical overwrite operation. + Overwrite, + /// Data files were removed and their contents logically deleted and/or delete files were added to delete rows. + Delete, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +/// Summarises the changes in the snapshot. +pub struct Summary { + /// The type of operation in the snapshot + pub operation: Operation, + /// Other summary data. + #[serde(flatten)] + pub other: HashMap, +} + +impl Default for Operation { + fn default() -> Operation { + Self::Append + } +} + +#[derive(Debug, PartialEq, Eq, Clone, Serialize, Deserialize, TypedBuilder)] +#[serde(from = "SnapshotV2", into = "SnapshotV2")] +#[builder(field_defaults(setter(prefix = "with_")))] +/// A snapshot represents the state of a table at some time and is used to access the complete set of data files in the table. +pub struct Snapshot { + /// A unique long ID + snapshot_id: i64, + /// The snapshot ID of the snapshot’s parent. + /// Omitted for any snapshot with no parent + #[builder(default = None)] + parent_snapshot_id: Option, + /// A monotonically increasing long that tracks the order of + /// changes to a table. + sequence_number: i64, + /// A timestamp when the snapshot was created, used for garbage + /// collection and table inspection + timestamp_ms: i64, + /// The location of a manifest list for this snapshot that + /// tracks manifest files with additional metadata. + /// Currently we only support manifest list file, and manifest files are not supported. + #[builder(setter(into))] + manifest_list: String, + /// A string map that summarizes the snapshot changes, including operation. + summary: Summary, + /// ID of the table’s current schema when the snapshot was created. + #[builder(setter(strip_option), default = None)] + schema_id: Option, +} + +impl Snapshot { + /// Get the id of the snapshot + #[inline] + pub fn snapshot_id(&self) -> i64 { + self.snapshot_id + } + + /// Get parent snapshot id. + #[inline] + pub fn parent_snapshot_id(&self) -> Option { + self.parent_snapshot_id + } + + /// Get sequence_number of the snapshot. Is 0 for Iceberg V1 tables. + #[inline] + pub fn sequence_number(&self) -> i64 { + self.sequence_number + } + /// Get location of manifest_list file + #[inline] + pub fn manifest_list(&self) -> &str { + &self.manifest_list + } + + /// Get summary of the snapshot + #[inline] + pub fn summary(&self) -> &Summary { + &self.summary + } + /// Get the timestamp of when the snapshot was created + #[inline] + pub fn timestamp(&self) -> DateTime { + Utc.timestamp_millis_opt(self.timestamp_ms).unwrap() + } + + /// Get the schema id of this snapshot. + #[inline] + pub fn schema_id(&self) -> Option { + self.schema_id + } + + /// Get the schema of this snapshot. + pub fn schema(&self, table_metadata: &TableMetadata) -> Result { + Ok(match self.schema_id() { + Some(schema_id) => table_metadata + .schema_by_id(schema_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Schema with id {} not found", schema_id), + ) + })? + .clone(), + None => table_metadata.current_schema().clone(), + }) + } + + /// Get parent snapshot. + #[cfg(test)] + pub(crate) fn parent_snapshot(&self, table_metadata: &TableMetadata) -> Option { + match self.parent_snapshot_id { + Some(id) => table_metadata.snapshot_by_id(id).cloned(), + None => None, + } + } + + /// Load manifest list. + pub async fn load_manifest_list( + &self, + file_io: &FileIO, + table_metadata: &TableMetadata, + ) -> Result { + let mut manifest_list_content = Vec::new(); + file_io + .new_input(&self.manifest_list)? + .reader() + .await? + .read_to_end(&mut manifest_list_content) + .await?; + + let schema = self.schema(table_metadata)?; + + let partition_type_provider = |partition_spec_id: i32| -> Result> { + table_metadata + .partition_spec_by_id(partition_spec_id) + .map(|partition_spec| partition_spec.partition_type(&schema)) + .transpose() + }; + + ManifestList::parse_with_version( + &manifest_list_content, + table_metadata.format_version(), + partition_type_provider, + ) + } + + pub(crate) fn log(&self) -> SnapshotLog { + SnapshotLog { + timestamp_ms: self.timestamp_ms, + snapshot_id: self.snapshot_id, + } + } +} + +pub(super) mod _serde { + /// This is a helper module that defines types to help with serialization/deserialization. + /// For deserialization the input first gets read into either the [SnapshotV1] or [SnapshotV2] struct + /// and then converted into the [Snapshot] struct. Serialization works the other way around. + /// [SnapshotV1] and [SnapshotV2] are internal struct that are only used for serialization and deserialization. + use std::collections::HashMap; + + use serde::{Deserialize, Serialize}; + + use crate::spec::SchemaId; + use crate::Error; + + use super::{Operation, Snapshot, Summary}; + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v2 snapshot for serialization/deserialization + pub(crate) struct SnapshotV2 { + pub snapshot_id: i64, + #[serde(skip_serializing_if = "Option::is_none")] + pub parent_snapshot_id: Option, + pub sequence_number: i64, + pub timestamp_ms: i64, + pub manifest_list: String, + pub summary: Summary, + #[serde(skip_serializing_if = "Option::is_none")] + pub schema_id: Option, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v1 snapshot for serialization/deserialization + pub(crate) struct SnapshotV1 { + pub snapshot_id: i64, + #[serde(skip_serializing_if = "Option::is_none")] + pub parent_snapshot_id: Option, + pub timestamp_ms: i64, + #[serde(skip_serializing_if = "Option::is_none")] + pub manifest_list: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub manifests: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub summary: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub schema_id: Option, + } + + impl From for Snapshot { + fn from(v2: SnapshotV2) -> Self { + Snapshot { + snapshot_id: v2.snapshot_id, + parent_snapshot_id: v2.parent_snapshot_id, + sequence_number: v2.sequence_number, + timestamp_ms: v2.timestamp_ms, + manifest_list: v2.manifest_list, + summary: v2.summary, + schema_id: v2.schema_id, + } + } + } + + impl From for SnapshotV2 { + fn from(v2: Snapshot) -> Self { + SnapshotV2 { + snapshot_id: v2.snapshot_id, + parent_snapshot_id: v2.parent_snapshot_id, + sequence_number: v2.sequence_number, + timestamp_ms: v2.timestamp_ms, + manifest_list: v2.manifest_list, + summary: v2.summary, + schema_id: v2.schema_id, + } + } + } + + impl TryFrom for Snapshot { + type Error = Error; + + fn try_from(v1: SnapshotV1) -> Result { + Ok(Snapshot { + snapshot_id: v1.snapshot_id, + parent_snapshot_id: v1.parent_snapshot_id, + sequence_number: 0, + timestamp_ms: v1.timestamp_ms, + manifest_list: match (v1.manifest_list, v1.manifests) { + (Some(file), None) => file, + (Some(_), Some(_)) => "Invalid v1 snapshot, when manifest list provided, manifest files should be omitted".to_string(), + (None, _) => "Unsupported v1 snapshot, only manifest list is supported".to_string() + }, + summary: v1.summary.unwrap_or(Summary { + operation: Operation::default(), + other: HashMap::new(), + }), + schema_id: v1.schema_id, + }) + } + } + + impl From for SnapshotV1 { + fn from(v2: Snapshot) -> Self { + SnapshotV1 { + snapshot_id: v2.snapshot_id, + parent_snapshot_id: v2.parent_snapshot_id, + timestamp_ms: v2.timestamp_ms, + manifest_list: Some(v2.manifest_list), + summary: Some(v2.summary), + schema_id: v2.schema_id, + manifests: None, + } + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +/// Iceberg tables keep track of branches and tags using snapshot references. +pub struct SnapshotReference { + /// A reference’s snapshot ID. The tagged snapshot or latest snapshot of a branch. + pub snapshot_id: i64, + #[serde(flatten)] + /// Snapshot retention policy + pub retention: SnapshotRetention, +} + +impl SnapshotReference { + /// Create new snapshot reference + pub fn new(snapshot_id: i64, retention: SnapshotRetention) -> Self { + SnapshotReference { + snapshot_id, + retention, + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "lowercase", tag = "type")] +/// The snapshot expiration procedure removes snapshots from table metadata and applies the table’s retention policy. +pub enum SnapshotRetention { + #[serde(rename_all = "kebab-case")] + /// Branches are mutable named references that can be updated by committing a new snapshot as + /// the branch’s referenced snapshot using the Commit Conflict Resolution and Retry procedures. + Branch { + /// A positive number for the minimum number of snapshots to keep in a branch while expiring snapshots. + /// Defaults to table property history.expire.min-snapshots-to-keep. + #[serde(skip_serializing_if = "Option::is_none")] + min_snapshots_to_keep: Option, + /// A positive number for the max age of snapshots to keep when expiring, including the latest snapshot. + /// Defaults to table property history.expire.max-snapshot-age-ms. + #[serde(skip_serializing_if = "Option::is_none")] + max_snapshot_age_ms: Option, + /// For snapshot references except the main branch, a positive number for the max age of the snapshot reference to keep while expiring snapshots. + /// Defaults to table property history.expire.max-ref-age-ms. The main branch never expires. + #[serde(skip_serializing_if = "Option::is_none")] + max_ref_age_ms: Option, + }, + #[serde(rename_all = "kebab-case")] + /// Tags are labels for individual snapshots. + Tag { + /// For snapshot references except the main branch, a positive number for the max age of the snapshot reference to keep while expiring snapshots. + /// Defaults to table property history.expire.max-ref-age-ms. The main branch never expires. + max_ref_age_ms: i64, + }, +} + +#[cfg(test)] +mod tests { + use chrono::{TimeZone, Utc}; + use std::collections::HashMap; + + use crate::spec::snapshot::{Operation, Snapshot, Summary, _serde::SnapshotV1}; + + #[test] + fn schema() { + let record = r#" + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://b/wh/.../s1.avro", + "schema-id": 0 + } + "#; + + let result: Snapshot = serde_json::from_str::(record) + .unwrap() + .try_into() + .unwrap(); + assert_eq!(3051729675574597004, result.snapshot_id()); + assert_eq!( + Utc.timestamp_millis_opt(1515100955770).unwrap(), + result.timestamp() + ); + assert_eq!( + Summary { + operation: Operation::Append, + other: HashMap::new() + }, + *result.summary() + ); + assert_eq!("s3://b/wh/.../s1.avro".to_string(), *result.manifest_list()); + } +} diff --git a/crates/iceberg/src/spec/sort.rs b/crates/iceberg/src/spec/sort.rs new file mode 100644 index 0000000..a4d2f7d --- /dev/null +++ b/crates/iceberg/src/spec/sort.rs @@ -0,0 +1,480 @@ +// 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. + +/*! + * Sorting +*/ +use crate::error::Result; +use crate::spec::Schema; +use crate::{Error, ErrorKind}; +use core::fmt; +use serde::{Deserialize, Serialize}; +use std::fmt::Formatter; +use std::sync::Arc; +use typed_builder::TypedBuilder; + +use super::transform::Transform; + +/// Reference to [`SortOrder`]. +pub type SortOrderRef = Arc; +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Copy, Clone)] +/// Sort direction in a partition, either ascending or descending +pub enum SortDirection { + /// Ascending + #[serde(rename = "asc")] + Ascending, + /// Descending + #[serde(rename = "desc")] + Descending, +} + +impl fmt::Display for SortDirection { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match *self { + SortDirection::Ascending => write!(f, "ascending"), + SortDirection::Descending => write!(f, "descending"), + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Copy, Clone)] +/// Describes the order of null values when sorted. +pub enum NullOrder { + #[serde(rename = "nulls-first")] + /// Nulls are stored first + First, + #[serde(rename = "nulls-last")] + /// Nulls are stored last + Last, +} + +impl fmt::Display for NullOrder { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match *self { + NullOrder::First => write!(f, "first"), + NullOrder::Last => write!(f, "last"), + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, TypedBuilder)] +#[serde(rename_all = "kebab-case")] +/// Entry for every column that is to be sorted +pub struct SortField { + /// A source column id from the table’s schema + pub source_id: i32, + /// A transform that is used to produce values to be sorted on from the source column. + pub transform: Transform, + /// A sort direction, that can only be either asc or desc + pub direction: SortDirection, + /// A null order that describes the order of null values when sorted. + pub null_order: NullOrder, +} + +impl fmt::Display for SortField { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "SortField {{ source_id: {}, transform: {}, direction: {}, null_order: {} }}", + self.source_id, self.transform, self.direction, self.null_order + ) + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Builder, Default)] +#[serde(rename_all = "kebab-case")] +#[builder(setter(prefix = "with"))] +#[builder(build_fn(skip))] +/// A sort order is defined by a sort order id and a list of sort fields. +/// The order of the sort fields within the list defines the order in which the sort is applied to the data. +pub struct SortOrder { + /// Identifier for SortOrder, order_id `0` is no sort order. + #[builder(default)] + pub order_id: i64, + /// Details of the sort + #[builder(setter(each(name = "with_sort_field")), default)] + pub fields: Vec, +} + +impl SortOrder { + const UNSORTED_ORDER_ID: i64 = 0; + + /// Create sort order builder + pub fn builder() -> SortOrderBuilder { + SortOrderBuilder::default() + } + + /// Create an unbound unsorted order + pub fn unsorted_order() -> SortOrder { + SortOrder { + order_id: SortOrder::UNSORTED_ORDER_ID, + fields: Vec::new(), + } + } + + /// Returns true if the sort order is unsorted. + /// + /// A [`SortOrder`] is unsorted if it has no sort fields. + pub fn is_unsorted(&self) -> bool { + self.fields.is_empty() + } +} + +impl SortOrderBuilder { + /// Creates a new unbound sort order. + pub fn build_unbound(&self) -> Result { + let fields = self.fields.clone().unwrap_or_default(); + return match (self.order_id, fields.as_slice()) { + (Some(SortOrder::UNSORTED_ORDER_ID) | None, []) => Ok(SortOrder::unsorted_order()), + (_, []) => Err(Error::new( + ErrorKind::Unexpected, + format!("Unsorted order ID must be {}", SortOrder::UNSORTED_ORDER_ID), + )), + (Some(SortOrder::UNSORTED_ORDER_ID), [..]) => Err(Error::new( + ErrorKind::Unexpected, + format!( + "Sort order ID {} is reserved for unsorted order", + SortOrder::UNSORTED_ORDER_ID + ), + )), + (maybe_order_id, [..]) => Ok(SortOrder { + order_id: maybe_order_id.unwrap_or(1), + fields: fields.to_vec(), + }), + }; + } + + /// Creates a new bound sort order. + pub fn build(&self, schema: Schema) -> Result { + let unbound_sort_order = self.build_unbound()?; + SortOrderBuilder::check_compatibility(unbound_sort_order, schema) + } + + /// Returns the given sort order if it is compatible with the given schema + fn check_compatibility(sort_order: SortOrder, schema: Schema) -> Result { + let sort_fields = &sort_order.fields; + for sort_field in sort_fields { + match schema.field_by_id(sort_field.source_id) { + None => { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Cannot find source column for sort field: {sort_field}"), + )) + } + Some(source_field) => { + let source_type = source_field.field_type.as_ref(); + + if !source_type.is_primitive() { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Cannot sort by non-primitive source field: {source_type}"), + )); + } + + let field_transform = sort_field.transform; + if field_transform.result_type(source_type).is_err() { + return Err(Error::new( + ErrorKind::Unexpected, + format!( + "Invalid source type {source_type} for transform {field_transform}" + ), + )); + } + } + } + } + + Ok(sort_order) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::spec::{ListType, NestedField, PrimitiveType, Type}; + + #[test] + fn test_sort_field() { + let spec = r#" + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + "#; + + let field: SortField = serde_json::from_str(spec).unwrap(); + assert_eq!(Transform::Bucket(4), field.transform); + assert_eq!(3, field.source_id); + assert_eq!(SortDirection::Descending, field.direction); + assert_eq!(NullOrder::Last, field.null_order); + } + + #[test] + fn test_sort_order() { + let spec = r#" + { + "order-id": 1, + "fields": [ { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } ] + } + "#; + + let order: SortOrder = serde_json::from_str(spec).unwrap(); + assert_eq!(Transform::Identity, order.fields[0].transform); + assert_eq!(2, order.fields[0].source_id); + assert_eq!(SortDirection::Ascending, order.fields[0].direction); + assert_eq!(NullOrder::First, order.fields[0].null_order); + + assert_eq!(Transform::Bucket(4), order.fields[1].transform); + assert_eq!(3, order.fields[1].source_id); + assert_eq!(SortDirection::Descending, order.fields[1].direction); + assert_eq!(NullOrder::Last, order.fields[1].null_order); + } + + #[test] + fn test_build_unbound_should_return_err_if_unsorted_order_does_not_have_an_order_id_of_zero() { + assert_eq!( + SortOrder::builder() + .with_order_id(1) + .build_unbound() + .expect_err("Expected an Err value") + .message(), + "Unsorted order ID must be 0" + ) + } + + #[test] + fn test_build_unbound_should_return_err_if_order_id_equals_zero_is_used_for_anything_other_than_unsorted_order( + ) { + assert_eq!( + SortOrder::builder() + .with_order_id(SortOrder::UNSORTED_ORDER_ID) + .with_sort_field( + SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build() + ) + .build_unbound() + .expect_err("Expected an Err value") + .message(), + "Sort order ID 0 is reserved for unsorted order" + ) + } + + #[test] + fn test_build_unbound_should_return_unsorted_sort_order() { + assert_eq!( + SortOrder::builder() + .with_order_id(SortOrder::UNSORTED_ORDER_ID) + .build_unbound() + .expect("Expected an Ok value"), + SortOrder::unsorted_order() + ) + } + + #[test] + fn test_build_unbound_should_return_sort_order_with_given_order_id_and_sort_fields() { + let sort_field = SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(); + + assert_eq!( + SortOrder::builder() + .with_order_id(2) + .with_sort_field(sort_field.clone()) + .build_unbound() + .expect("Expected an Ok value"), + SortOrder { + order_id: 2, + fields: vec![sort_field] + } + ) + } + + #[test] + fn test_build_unbound_should_return_sort_order_with_given_sort_fields_and_defaults_to_1_if_missing_an_order_id( + ) { + let sort_field = SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(); + + assert_eq!( + SortOrder::builder() + .with_sort_field(sort_field.clone()) + .build_unbound() + .expect("Expected an Ok value"), + SortOrder { + order_id: 1, + fields: vec![sort_field] + } + ) + } + + #[test] + fn test_build_should_return_err_if_sort_order_field_is_not_present_in_schema() { + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![NestedField::required( + 1, + "foo", + Type::Primitive(PrimitiveType::Int), + ) + .into()]) + .build() + .unwrap(); + + let sort_order_builder_result = SortOrder::builder() + .with_sort_field( + SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(), + ) + .build(schema); + + assert_eq!( + sort_order_builder_result + .expect_err("Expected an Err value") + .message(), + "Cannot find source column for sort field: SortField { source_id: 2, transform: identity, direction: ascending, null_order: first }" + ) + } + + #[test] + fn test_build_should_return_err_if_source_field_is_not_a_primitive_type() { + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![NestedField::required( + 1, + "foo", + Type::List(ListType { + element_field: NestedField::list_element( + 2, + Type::Primitive(PrimitiveType::String), + true, + ) + .into(), + }), + ) + .into()]) + .build() + .unwrap(); + + let sort_order_builder_result = SortOrder::builder() + .with_sort_field( + SortField::builder() + .source_id(1) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(), + ) + .build(schema); + + assert_eq!( + sort_order_builder_result + .expect_err("Expected an Err value") + .message(), + "Cannot sort by non-primitive source field: list" + ) + } + + #[test] + fn test_build_should_return_err_if_source_field_type_is_not_supported_by_transform() { + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![NestedField::required( + 1, + "foo", + Type::Primitive(PrimitiveType::Int), + ) + .into()]) + .build() + .unwrap(); + + let sort_order_builder_result = SortOrder::builder() + .with_sort_field( + SortField::builder() + .source_id(1) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Year) + .build(), + ) + .build(schema); + + assert_eq!( + sort_order_builder_result + .expect_err("Expected an Err value") + .message(), + "Invalid source type int for transform year" + ) + } + + #[test] + fn test_build_should_return_valid_sort_order() { + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(); + + let sort_field = SortField::builder() + .source_id(2) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .transform(Transform::Identity) + .build(); + + let sort_order_builder_result = SortOrder::builder() + .with_sort_field(sort_field.clone()) + .build(schema); + + assert_eq!( + sort_order_builder_result.expect("Expected an Ok value"), + SortOrder { + order_id: 1, + fields: vec![sort_field], + } + ) + } +} diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs new file mode 100644 index 0000000..a6eb05c --- /dev/null +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -0,0 +1,1572 @@ +// 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. + +//! Defines the [table metadata](https://iceberg.apache.org/spec/#table-metadata). +//! The main struct here is [TableMetadataV2] which defines the data for a table. + +use serde::{Deserialize, Serialize}; +use serde_repr::{Deserialize_repr, Serialize_repr}; +use std::cmp::Ordering; +use std::fmt::{Display, Formatter}; +use std::{collections::HashMap, sync::Arc}; +use uuid::Uuid; + +use super::{ + snapshot::{Snapshot, SnapshotReference, SnapshotRetention}, + PartitionSpecRef, SchemaId, SchemaRef, SnapshotRef, SortOrderRef, +}; + +use _serde::TableMetadataEnum; + +use chrono::{DateTime, TimeZone, Utc}; + +static MAIN_BRANCH: &str = "main"; +static DEFAULT_SPEC_ID: i32 = 0; +static DEFAULT_SORT_ORDER_ID: i64 = 0; + +pub(crate) static EMPTY_SNAPSHOT_ID: i64 = -1; +pub(crate) static INITIAL_SEQUENCE_NUMBER: i64 = 0; + +/// Reference to [`TableMetadata`]. +pub type TableMetadataRef = Arc; + +#[derive(Debug, PartialEq, Deserialize, Eq, Clone)] +#[serde(try_from = "TableMetadataEnum")] +/// Fields for the version 2 of the table metadata. +/// +/// We assume that this data structure is always valid, so we will panic when invalid error happens. +/// We check the validity of this data structure when constructing. +pub struct TableMetadata { + /// Integer Version for the format. + format_version: FormatVersion, + /// A UUID that identifies the table + table_uuid: Uuid, + /// Location tables base location + location: String, + /// The tables highest sequence number + last_sequence_number: i64, + /// Timestamp in milliseconds from the unix epoch when the table was last updated. + last_updated_ms: i64, + /// An integer; the highest assigned column ID for the table. + last_column_id: i32, + /// A list of schemas, stored as objects with schema-id. + schemas: HashMap, + /// ID of the table’s current schema. + current_schema_id: i32, + /// A list of partition specs, stored as full partition spec objects. + partition_specs: HashMap, + /// ID of the “current” spec that writers should use by default. + default_spec_id: i32, + /// An integer; the highest assigned partition field ID across all partition specs for the table. + last_partition_id: i32, + ///A string to string map of table properties. This is used to control settings that + /// affect reading and writing and is not intended to be used for arbitrary metadata. + /// For example, commit.retry.num-retries is used to control the number of commit retries. + properties: HashMap, + /// long ID of the current table snapshot; must be the same as the current + /// ID of the main branch in refs. + current_snapshot_id: Option, + ///A list of valid snapshots. Valid snapshots are snapshots for which all + /// data files exist in the file system. A data file must not be deleted + /// from the file system until the last snapshot in which it was listed is + /// garbage collected. + snapshots: HashMap, + /// A list (optional) of timestamp and snapshot ID pairs that encodes changes + /// to the current snapshot for the table. Each time the current-snapshot-id + /// is changed, a new entry should be added with the last-updated-ms + /// and the new current-snapshot-id. When snapshots are expired from + /// the list of valid snapshots, all entries before a snapshot that has + /// expired should be removed. + snapshot_log: Vec, + + /// A list (optional) of timestamp and metadata file location pairs + /// that encodes changes to the previous metadata files for the table. + /// Each time a new metadata file is created, a new entry of the + /// previous metadata file location should be added to the list. + /// Tables can be configured to remove oldest metadata log entries and + /// keep a fixed-size log of the most recent entries after a commit. + metadata_log: Vec, + + /// A list of sort orders, stored as full sort order objects. + sort_orders: HashMap, + /// Default sort order id of the table. Note that this could be used by + /// writers, but is not used when reading because reads use the specs + /// stored in manifest files. + default_sort_order_id: i64, + ///A map of snapshot references. The map keys are the unique snapshot reference + /// names in the table, and the map values are snapshot reference objects. + /// There is always a main branch reference pointing to the current-snapshot-id + /// even if the refs map is null. + refs: HashMap, +} + +impl TableMetadata { + /// Returns format version of this metadata. + #[inline] + pub fn format_version(&self) -> FormatVersion { + self.format_version + } + + /// Returns uuid of current table. + #[inline] + pub fn uuid(&self) -> Uuid { + self.table_uuid + } + + /// Returns table location. + #[inline] + pub fn location(&self) -> &str { + self.location.as_str() + } + + /// Returns last sequence number. + #[inline] + pub fn last_sequence_number(&self) -> i64 { + self.last_sequence_number + } + + /// Returns last updated time. + #[inline] + pub fn last_updated_ms(&self) -> DateTime { + Utc.timestamp_millis_opt(self.last_updated_ms).unwrap() + } + + /// Returns schemas + #[inline] + pub fn schemas_iter(&self) -> impl Iterator { + self.schemas.values() + } + + /// Lookup schema by id. + #[inline] + pub fn schema_by_id(&self, schema_id: SchemaId) -> Option<&SchemaRef> { + self.schemas.get(&schema_id) + } + + /// Get current schema + #[inline] + pub fn current_schema(&self) -> &SchemaRef { + self.schema_by_id(self.current_schema_id) + .expect("Current schema id set, but not found in table metadata") + } + + /// Returns all partition specs. + #[inline] + pub fn partition_specs_iter(&self) -> impl Iterator { + self.partition_specs.values() + } + + /// Lookup partition spec by id. + #[inline] + pub fn partition_spec_by_id(&self, spec_id: i32) -> Option<&PartitionSpecRef> { + self.partition_specs.get(&spec_id) + } + + /// Get default partition spec + #[inline] + pub fn default_partition_spec(&self) -> Option<&PartitionSpecRef> { + if self.default_spec_id == DEFAULT_SPEC_ID { + self.partition_spec_by_id(DEFAULT_SPEC_ID) + } else { + Some( + self.partition_spec_by_id(self.default_spec_id) + .expect("Default partition spec id set, but not found in table metadata"), + ) + } + } + + /// Returns all snapshots + #[inline] + pub fn snapshots(&self) -> impl Iterator { + self.snapshots.values() + } + + /// Lookup snapshot by id. + #[inline] + pub fn snapshot_by_id(&self, snapshot_id: i64) -> Option<&SnapshotRef> { + self.snapshots.get(&snapshot_id) + } + + /// Returns snapshot history. + #[inline] + pub fn history(&self) -> &[SnapshotLog] { + &self.snapshot_log + } + + /// Get current snapshot + #[inline] + pub fn current_snapshot(&self) -> Option<&SnapshotRef> { + self.current_snapshot_id.map(|s| { + self.snapshot_by_id(s) + .expect("Current snapshot id has been set, but doesn't exist in metadata") + }) + } + + /// Return all sort orders. + #[inline] + pub fn sort_orders_iter(&self) -> impl Iterator { + self.sort_orders.values() + } + + /// Lookup sort order by id. + #[inline] + pub fn sort_order_by_id(&self, sort_order_id: i64) -> Option<&SortOrderRef> { + self.sort_orders.get(&sort_order_id) + } + + /// Returns default sort order id. + #[inline] + pub fn default_sort_order(&self) -> Option<&SortOrderRef> { + if self.default_sort_order_id == DEFAULT_SORT_ORDER_ID { + self.sort_orders.get(&DEFAULT_SORT_ORDER_ID) + } else { + Some( + self.sort_orders + .get(&self.default_sort_order_id) + .expect("Default order id has been set, but not found in table metadata!"), + ) + } + } + + /// Returns properties of table. + #[inline] + pub fn properties(&self) -> &HashMap { + &self.properties + } + + /// Append snapshot to table + pub fn append_snapshot(&mut self, snapshot: Snapshot) { + self.last_updated_ms = snapshot.timestamp().timestamp_millis(); + self.last_sequence_number = snapshot.sequence_number(); + + self.refs + .entry(MAIN_BRANCH.to_string()) + .and_modify(|s| { + s.snapshot_id = snapshot.snapshot_id(); + }) + .or_insert_with(|| { + SnapshotReference::new( + snapshot.snapshot_id(), + SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + ) + }); + + self.snapshot_log.push(snapshot.log()); + self.snapshots + .insert(snapshot.snapshot_id(), Arc::new(snapshot)); + } +} + +pub(super) mod _serde { + /// This is a helper module that defines types to help with serialization/deserialization. + /// For deserialization the input first gets read into either the [TableMetadataV1] or [TableMetadataV2] struct + /// and then converted into the [TableMetadata] struct. Serialization works the other way around. + /// [TableMetadataV1] and [TableMetadataV2] are internal struct that are only used for serialization and deserialization. + use std::{collections::HashMap, sync::Arc}; + + use itertools::Itertools; + use serde::{Deserialize, Serialize}; + use uuid::Uuid; + + use crate::spec::{Snapshot, EMPTY_SNAPSHOT_ID}; + use crate::{ + spec::{ + schema::_serde::{SchemaV1, SchemaV2}, + snapshot::_serde::{SnapshotV1, SnapshotV2}, + PartitionField, PartitionSpec, Schema, SnapshotReference, SnapshotRetention, SortOrder, + }, + Error, ErrorKind, + }; + + use super::{ + FormatVersion, MetadataLog, SnapshotLog, TableMetadata, DEFAULT_SORT_ORDER_ID, + DEFAULT_SPEC_ID, MAIN_BRANCH, + }; + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(untagged)] + pub(super) enum TableMetadataEnum { + V2(TableMetadataV2), + V1(TableMetadataV1), + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v2 table metadata for serialization/deserialization + pub(super) struct TableMetadataV2 { + pub format_version: VersionNumber<2>, + pub table_uuid: Uuid, + pub location: String, + pub last_sequence_number: i64, + pub last_updated_ms: i64, + pub last_column_id: i32, + pub schemas: Vec, + pub current_schema_id: i32, + pub partition_specs: Vec, + pub default_spec_id: i32, + pub last_partition_id: i32, + #[serde(skip_serializing_if = "Option::is_none")] + pub properties: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub current_snapshot_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub snapshots: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub snapshot_log: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub metadata_log: Option>, + pub sort_orders: Vec, + pub default_sort_order_id: i64, + #[serde(skip_serializing_if = "Option::is_none")] + pub refs: Option>, + } + + #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v1 table metadata for serialization/deserialization + pub(super) struct TableMetadataV1 { + pub format_version: VersionNumber<1>, + #[serde(skip_serializing_if = "Option::is_none")] + pub table_uuid: Option, + pub location: String, + pub last_updated_ms: i64, + pub last_column_id: i32, + pub schema: SchemaV1, + #[serde(skip_serializing_if = "Option::is_none")] + pub schemas: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub current_schema_id: Option, + pub partition_spec: Vec, + #[serde(skip_serializing_if = "Option::is_none")] + pub partition_specs: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub default_spec_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub last_partition_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub properties: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub current_snapshot_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub snapshots: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub snapshot_log: Option>, + #[serde(skip_serializing_if = "Option::is_none")] + pub metadata_log: Option>, + pub sort_orders: Option>, + pub default_sort_order_id: Option, + } + + /// Helper to serialize and deserialize the format version. + #[derive(Debug, PartialEq, Eq)] + pub(super) struct VersionNumber; + + impl Serialize for TableMetadata { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + // we must do a clone here + let table_metadata_enum: TableMetadataEnum = + self.clone().try_into().map_err(serde::ser::Error::custom)?; + + table_metadata_enum.serialize(serializer) + } + } + + impl Serialize for VersionNumber { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + serializer.serialize_u8(V) + } + } + + impl<'de, const V: u8> Deserialize<'de> for VersionNumber { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + let value = u8::deserialize(deserializer)?; + if value == V { + Ok(VersionNumber::) + } else { + Err(serde::de::Error::custom("Invalid Version")) + } + } + } + + impl TryFrom for TableMetadata { + type Error = Error; + fn try_from(value: TableMetadataEnum) -> Result { + match value { + TableMetadataEnum::V2(value) => value.try_into(), + TableMetadataEnum::V1(value) => value.try_into(), + } + } + } + + impl TryFrom for TableMetadataEnum { + type Error = Error; + fn try_from(value: TableMetadata) -> Result { + Ok(match value.format_version { + FormatVersion::V2 => TableMetadataEnum::V2(value.into()), + FormatVersion::V1 => TableMetadataEnum::V1(value.try_into()?), + }) + } + } + + impl TryFrom for TableMetadata { + type Error = Error; + fn try_from(value: TableMetadataV2) -> Result { + let current_snapshot_id = if let &Some(-1) = &value.current_snapshot_id { + None + } else { + value.current_snapshot_id + }; + let schemas = HashMap::from_iter( + value + .schemas + .into_iter() + .map(|schema| Ok((schema.schema_id, Arc::new(schema.try_into()?)))) + .collect::, Error>>()?, + ); + Ok(TableMetadata { + format_version: FormatVersion::V2, + table_uuid: value.table_uuid, + location: value.location, + last_sequence_number: value.last_sequence_number, + last_updated_ms: value.last_updated_ms, + last_column_id: value.last_column_id, + current_schema_id: if schemas.keys().contains(&value.current_schema_id) { + Ok(value.current_schema_id) + } else { + Err(self::Error::new( + ErrorKind::DataInvalid, + format!( + "No schema exists with the current schema id {}.", + value.current_schema_id + ), + )) + }?, + schemas, + partition_specs: HashMap::from_iter( + value + .partition_specs + .into_iter() + .map(|x| (x.spec_id, Arc::new(x))), + ), + default_spec_id: value.default_spec_id, + last_partition_id: value.last_partition_id, + properties: value.properties.unwrap_or_default(), + current_snapshot_id, + snapshots: value + .snapshots + .map(|snapshots| { + HashMap::from_iter( + snapshots + .into_iter() + .map(|x| (x.snapshot_id, Arc::new(x.into()))), + ) + }) + .unwrap_or_default(), + snapshot_log: value.snapshot_log.unwrap_or_default(), + metadata_log: value.metadata_log.unwrap_or_default(), + sort_orders: HashMap::from_iter( + value + .sort_orders + .into_iter() + .map(|x| (x.order_id, Arc::new(x))), + ), + default_sort_order_id: value.default_sort_order_id, + refs: value.refs.unwrap_or_else(|| { + if let Some(snapshot_id) = current_snapshot_id { + HashMap::from_iter(vec![( + MAIN_BRANCH.to_string(), + SnapshotReference { + snapshot_id, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + }, + )]) + } else { + HashMap::new() + } + }), + }) + } + } + + impl TryFrom for TableMetadata { + type Error = Error; + fn try_from(value: TableMetadataV1) -> Result { + let schemas = value + .schemas + .map(|schemas| { + Ok::<_, Error>(HashMap::from_iter( + schemas + .into_iter() + .enumerate() + .map(|(i, schema)| { + Ok(( + schema.schema_id.unwrap_or(i as i32), + Arc::new(schema.try_into()?), + )) + }) + .collect::, Error>>()? + .into_iter(), + )) + }) + .or_else(|| { + Some(value.schema.try_into().map(|schema: Schema| { + HashMap::from_iter(vec![(schema.schema_id(), Arc::new(schema))]) + })) + }) + .transpose()? + .unwrap_or_default(); + let partition_specs = HashMap::from_iter( + value + .partition_specs + .unwrap_or_else(|| { + vec![PartitionSpec { + spec_id: DEFAULT_SPEC_ID, + fields: value.partition_spec, + }] + }) + .into_iter() + .map(|x| (x.spec_id, Arc::new(x))), + ); + Ok(TableMetadata { + format_version: FormatVersion::V1, + table_uuid: value.table_uuid.unwrap_or_default(), + location: value.location, + last_sequence_number: 0, + last_updated_ms: value.last_updated_ms, + last_column_id: value.last_column_id, + current_schema_id: value + .current_schema_id + .unwrap_or_else(|| schemas.keys().copied().max().unwrap_or_default()), + default_spec_id: value + .default_spec_id + .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()), + last_partition_id: value + .last_partition_id + .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()), + partition_specs, + schemas, + + properties: value.properties.unwrap_or_default(), + current_snapshot_id: if let &Some(id) = &value.current_snapshot_id { + if id == EMPTY_SNAPSHOT_ID { + None + } else { + Some(id) + } + } else { + value.current_snapshot_id + }, + snapshots: value + .snapshots + .map(|snapshots| { + Ok::<_, Error>(HashMap::from_iter( + snapshots + .into_iter() + .map(|x| Ok((x.snapshot_id, Arc::new(x.try_into()?)))) + .collect::, Error>>()?, + )) + }) + .transpose()? + .unwrap_or_default(), + snapshot_log: value.snapshot_log.unwrap_or_default(), + metadata_log: value.metadata_log.unwrap_or_default(), + sort_orders: match value.sort_orders { + Some(sort_orders) => HashMap::from_iter( + sort_orders.into_iter().map(|x| (x.order_id, Arc::new(x))), + ), + None => HashMap::new(), + }, + default_sort_order_id: value.default_sort_order_id.unwrap_or(DEFAULT_SORT_ORDER_ID), + refs: HashMap::from_iter(vec![( + MAIN_BRANCH.to_string(), + SnapshotReference { + snapshot_id: value.current_snapshot_id.unwrap_or_default(), + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + }, + )]), + }) + } + } + + impl From for TableMetadataV2 { + fn from(v: TableMetadata) -> Self { + TableMetadataV2 { + format_version: VersionNumber::<2>, + table_uuid: v.table_uuid, + location: v.location, + last_sequence_number: v.last_sequence_number, + last_updated_ms: v.last_updated_ms, + last_column_id: v.last_column_id, + schemas: v + .schemas + .into_values() + .map(|x| { + Arc::try_unwrap(x) + .unwrap_or_else(|schema| schema.as_ref().clone()) + .into() + }) + .collect(), + current_schema_id: v.current_schema_id, + partition_specs: v + .partition_specs + .into_values() + .map(|x| Arc::try_unwrap(x).unwrap_or_else(|s| s.as_ref().clone())) + .collect(), + default_spec_id: v.default_spec_id, + last_partition_id: v.last_partition_id, + properties: if v.properties.is_empty() { + None + } else { + Some(v.properties) + }, + current_snapshot_id: v.current_snapshot_id.or(Some(-1)), + snapshots: if v.snapshots.is_empty() { + None + } else { + Some( + v.snapshots + .into_values() + .map(|x| { + Arc::try_unwrap(x) + .unwrap_or_else(|snapshot| snapshot.as_ref().clone()) + .into() + }) + .collect(), + ) + }, + snapshot_log: if v.snapshot_log.is_empty() { + None + } else { + Some(v.snapshot_log) + }, + metadata_log: if v.metadata_log.is_empty() { + None + } else { + Some(v.metadata_log) + }, + sort_orders: v + .sort_orders + .into_values() + .map(|x| Arc::try_unwrap(x).unwrap_or_else(|s| s.as_ref().clone())) + .collect(), + default_sort_order_id: v.default_sort_order_id, + refs: Some(v.refs), + } + } + } + + impl TryFrom for TableMetadataV1 { + type Error = Error; + fn try_from(v: TableMetadata) -> Result { + Ok(TableMetadataV1 { + format_version: VersionNumber::<1>, + table_uuid: Some(v.table_uuid), + location: v.location, + last_updated_ms: v.last_updated_ms, + last_column_id: v.last_column_id, + schema: v + .schemas + .get(&v.current_schema_id) + .ok_or(Error::new( + ErrorKind::Unexpected, + "current_schema_id not found in schemas", + ))? + .as_ref() + .clone() + .into(), + schemas: Some( + v.schemas + .into_values() + .map(|x| { + Arc::try_unwrap(x) + .unwrap_or_else(|schema| schema.as_ref().clone()) + .into() + }) + .collect(), + ), + current_schema_id: Some(v.current_schema_id), + partition_spec: v + .partition_specs + .get(&v.default_spec_id) + .map(|x| x.fields.clone()) + .unwrap_or_default(), + partition_specs: Some( + v.partition_specs + .into_values() + .map(|x| Arc::try_unwrap(x).unwrap_or_else(|s| s.as_ref().clone())) + .collect(), + ), + default_spec_id: Some(v.default_spec_id), + last_partition_id: Some(v.last_partition_id), + properties: if v.properties.is_empty() { + None + } else { + Some(v.properties) + }, + current_snapshot_id: v.current_snapshot_id.or(Some(-1)), + snapshots: if v.snapshots.is_empty() { + None + } else { + Some( + v.snapshots + .into_values() + .map(|x| Snapshot::clone(&x).into()) + .collect(), + ) + }, + snapshot_log: if v.snapshot_log.is_empty() { + None + } else { + Some(v.snapshot_log) + }, + metadata_log: if v.metadata_log.is_empty() { + None + } else { + Some(v.metadata_log) + }, + sort_orders: Some( + v.sort_orders + .into_values() + .map(|s| Arc::try_unwrap(s).unwrap_or_else(|s| s.as_ref().clone())) + .collect(), + ), + default_sort_order_id: Some(v.default_sort_order_id), + }) + } + } +} + +#[derive(Debug, Serialize_repr, Deserialize_repr, PartialEq, Eq, Clone, Copy)] +#[repr(u8)] +/// Iceberg format version +pub enum FormatVersion { + /// Iceberg spec version 1 + V1 = 1u8, + /// Iceberg spec version 2 + V2 = 2u8, +} + +impl PartialOrd for FormatVersion { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for FormatVersion { + fn cmp(&self, other: &Self) -> Ordering { + (*self as u8).cmp(&(*other as u8)) + } +} + +impl Display for FormatVersion { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + FormatVersion::V1 => write!(f, "v1"), + FormatVersion::V2 => write!(f, "v2"), + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +/// Encodes changes to the previous metadata files for the table +pub struct MetadataLog { + /// The file for the log. + pub metadata_file: String, + /// Time new metadata was created + pub timestamp_ms: i64, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +/// A log of when each snapshot was made. +pub struct SnapshotLog { + /// Id of the snapshot. + pub snapshot_id: i64, + /// Last updated timestamp + pub timestamp_ms: i64, +} + +impl SnapshotLog { + /// Returns the last updated timestamp as a DateTime with millisecond precision + pub fn timestamp(self) -> DateTime { + Utc.timestamp_millis_opt(self.timestamp_ms).unwrap() + } +} + +#[cfg(test)] +mod tests { + use std::{collections::HashMap, fs, sync::Arc}; + + use anyhow::Result; + use uuid::Uuid; + + use pretty_assertions::assert_eq; + + use crate::spec::{ + table_metadata::TableMetadata, NestedField, NullOrder, Operation, PartitionField, + PartitionSpec, PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, + SortDirection, SortField, SortOrder, Summary, Transform, Type, + }; + + use super::{FormatVersion, MetadataLog, SnapshotLog}; + + fn check_table_metadata_serde(json: &str, expected_type: TableMetadata) { + let desered_type: TableMetadata = serde_json::from_str(json).unwrap(); + assert_eq!(desered_type, expected_type); + + let sered_json = serde_json::to_string(&expected_type).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + + assert_eq!(parsed_json_value, desered_type); + } + + fn get_test_table_metadata(file_name: &str) -> TableMetadata { + let path = format!("testdata/table_metadata/{}", file_name); + let metadata: String = fs::read_to_string(path).unwrap(); + + serde_json::from_str(&metadata).unwrap() + } + + #[test] + fn test_table_data_v2() { + let data = r#" + { + "format-version" : 2, + "table-uuid": "fb072c92-a02b-11e9-ae9c-1bb7bc9eca94", + "location": "s3://b/wh/data.db/table", + "last-sequence-number" : 1, + "last-updated-ms": 1515100955770, + "last-column-id": 1, + "schemas": [ + { + "schema-id" : 1, + "type" : "struct", + "fields" :[ + { + "id": 1, + "name": "struct_name", + "required": true, + "type": "fixed[1]" + } + ] + } + ], + "current-schema-id" : 1, + "partition-specs": [ + { + "spec-id": 1, + "fields": [ + { + "source-id": 4, + "field-id": 1000, + "name": "ts_day", + "transform": "day" + } + ] + } + ], + "default-spec-id": 1, + "last-partition-id": 1000, + "properties": { + "commit.retry.num-retries": "1" + }, + "metadata-log": [ + { + "metadata-file": "s3://bucket/.../v1.json", + "timestamp-ms": 1515100 + } + ], + "sort-orders": [], + "default-sort-order-id": 0 + } + "#; + + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![Arc::new(NestedField::required( + 1, + "struct_name", + Type::Primitive(PrimitiveType::Fixed(1)), + ))]) + .build() + .unwrap(); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(1) + .with_partition_field(PartitionField { + name: "ts_day".to_string(), + transform: Transform::Day, + source_id: 4, + field_id: 1000, + }) + .build() + .unwrap(); + + let expected = TableMetadata { + format_version: FormatVersion::V2, + table_uuid: Uuid::parse_str("fb072c92-a02b-11e9-ae9c-1bb7bc9eca94").unwrap(), + location: "s3://b/wh/data.db/table".to_string(), + last_updated_ms: 1515100955770, + last_column_id: 1, + schemas: HashMap::from_iter(vec![(1, Arc::new(schema))]), + current_schema_id: 1, + partition_specs: HashMap::from_iter(vec![(1, partition_spec.into())]), + default_spec_id: 1, + last_partition_id: 1000, + default_sort_order_id: 0, + sort_orders: HashMap::from_iter(vec![]), + snapshots: HashMap::default(), + current_snapshot_id: None, + last_sequence_number: 1, + properties: HashMap::from_iter(vec![( + "commit.retry.num-retries".to_string(), + "1".to_string(), + )]), + snapshot_log: Vec::new(), + metadata_log: vec![MetadataLog { + metadata_file: "s3://bucket/.../v1.json".to_string(), + timestamp_ms: 1515100, + }], + refs: HashMap::new(), + }; + + check_table_metadata_serde(data, expected); + } + + #[test] + fn test_table_data_v1() { + let data = r#" + { + "format-version" : 1, + "table-uuid" : "df838b92-0b32-465d-a44e-d39936e538b7", + "location" : "/home/iceberg/warehouse/nyc/taxis", + "last-updated-ms" : 1662532818843, + "last-column-id" : 5, + "schema" : { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "vendor_id", + "required" : false, + "type" : "long" + }, { + "id" : 2, + "name" : "trip_id", + "required" : false, + "type" : "long" + }, { + "id" : 3, + "name" : "trip_distance", + "required" : false, + "type" : "float" + }, { + "id" : 4, + "name" : "fare_amount", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "store_and_fwd_flag", + "required" : false, + "type" : "string" + } ] + }, + "partition-spec" : [ { + "name" : "vendor_id", + "transform" : "identity", + "source-id" : 1, + "field-id" : 1000 + } ], + "last-partition-id" : 1000, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root" + }, + "current-snapshot-id" : 638933773299822130, + "refs" : { + "main" : { + "snapshot-id" : 638933773299822130, + "type" : "branch" + } + }, + "snapshots" : [ { + "snapshot-id" : 638933773299822130, + "timestamp-ms" : 1662532818843, + "sequence-number" : 0, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1662532784305", + "added-data-files" : "4", + "added-records" : "4", + "added-files-size" : "6001" + }, + "manifest-list" : "/home/iceberg/warehouse/nyc/taxis/metadata/snap-638933773299822130-1-7e6760f0-4f6c-4b23-b907-0a5a174e3863.avro", + "schema-id" : 0 + } ], + "snapshot-log" : [ { + "timestamp-ms" : 1662532818843, + "snapshot-id" : 638933773299822130 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1662532805245, + "metadata-file" : "/home/iceberg/warehouse/nyc/taxis/metadata/00000-8a62c37d-4573-4021-952a-c0baef7d21d0.metadata.json" + } ] + } + "#; + + let schema = Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "vendor_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "trip_id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 3, + "trip_distance", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 4, + "fare_amount", + Type::Primitive(PrimitiveType::Double), + )), + Arc::new(NestedField::optional( + 5, + "store_and_fwd_flag", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(0) + .with_partition_field(PartitionField { + name: "vendor_id".to_string(), + transform: Transform::Identity, + source_id: 1, + field_id: 1000, + }) + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(0) + .build_unbound() + .unwrap(); + + let snapshot = Snapshot::builder() + .with_snapshot_id(638933773299822130) + .with_timestamp_ms(1662532818843) + .with_sequence_number(0) + .with_schema_id(0) + .with_manifest_list("/home/iceberg/warehouse/nyc/taxis/metadata/snap-638933773299822130-1-7e6760f0-4f6c-4b23-b907-0a5a174e3863.avro") + .with_summary(Summary { operation: Operation::Append, other: HashMap::from_iter(vec![("spark.app.id".to_string(), "local-1662532784305".to_string()), ("added-data-files".to_string(), "4".to_string()), ("added-records".to_string(), "4".to_string()), ("added-files-size".to_string(), "6001".to_string())]) }) + .build(); + + let expected = TableMetadata { + format_version: FormatVersion::V1, + table_uuid: Uuid::parse_str("df838b92-0b32-465d-a44e-d39936e538b7").unwrap(), + location: "/home/iceberg/warehouse/nyc/taxis".to_string(), + last_updated_ms: 1662532818843, + last_column_id: 5, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + current_schema_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), + default_spec_id: 0, + last_partition_id: 1000, + default_sort_order_id: 0, + sort_orders: HashMap::from_iter(vec![(0, sort_order.into())]), + snapshots: HashMap::from_iter(vec![(638933773299822130, Arc::new(snapshot))]), + current_snapshot_id: Some(638933773299822130), + last_sequence_number: 0, + properties: HashMap::from_iter(vec![("owner".to_string(), "root".to_string())]), + snapshot_log: vec![SnapshotLog { + snapshot_id: 638933773299822130, + timestamp_ms: 1662532818843, + }], + metadata_log: vec![MetadataLog { metadata_file: "/home/iceberg/warehouse/nyc/taxis/metadata/00000-8a62c37d-4573-4021-952a-c0baef7d21d0.metadata.json".to_string(), timestamp_ms: 1662532805245 }], + refs: HashMap::from_iter(vec![("main".to_string(), SnapshotReference { snapshot_id: 638933773299822130, retention: SnapshotRetention::Branch { min_snapshots_to_keep: None, max_snapshot_age_ms: None, max_ref_age_ms: None } })]), + }; + + check_table_metadata_serde(data, expected); + } + + #[test] + fn test_invalid_table_uuid() -> Result<()> { + let data = r#" + { + "format-version" : 2, + "table-uuid": "xxxx" + } + "#; + assert!(serde_json::from_str::(data).is_err()); + Ok(()) + } + + #[test] + fn test_deserialize_table_data_v2_invalid_format_version() -> Result<()> { + let data = r#" + { + "format-version" : 1 + } + "#; + assert!(serde_json::from_str::(data).is_err()); + Ok(()) + } + + #[test] + fn test_table_metadata_v2_file_valid() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2Valid.json").unwrap(); + + let schema1 = Schema::builder() + .with_schema_id(0) + .with_fields(vec![Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + ))]) + .build() + .unwrap(); + + let schema2 = Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new( + NestedField::required(2, "y", Type::Primitive(PrimitiveType::Long)) + .with_doc("comment"), + ), + Arc::new(NestedField::required( + 3, + "z", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .with_identifier_field_ids(vec![1, 2]) + .build() + .unwrap(); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(0) + .with_partition_field(PartitionField { + name: "x".to_string(), + transform: Transform::Identity, + source_id: 1, + field_id: 1000, + }) + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(3) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Identity, + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .with_sort_field(SortField { + source_id: 3, + transform: Transform::Bucket(4), + direction: SortDirection::Descending, + null_order: NullOrder::Last, + }) + .build_unbound() + .unwrap(); + + let snapshot1 = Snapshot::builder() + .with_snapshot_id(3051729675574597004) + .with_timestamp_ms(1515100955770) + .with_sequence_number(0) + .with_manifest_list("s3://a/b/1.avro") + .with_summary(Summary { + operation: Operation::Append, + other: HashMap::new(), + }) + .build(); + + let snapshot2 = Snapshot::builder() + .with_snapshot_id(3055729675574597004) + .with_parent_snapshot_id(Some(3051729675574597004)) + .with_timestamp_ms(1555100955770) + .with_sequence_number(1) + .with_schema_id(1) + .with_manifest_list("s3://a/b/2.avro") + .with_summary(Summary { + operation: Operation::Append, + other: HashMap::new(), + }) + .build(); + + let expected = TableMetadata { + format_version: FormatVersion::V2, + table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), + location: "s3://bucket/test/location".to_string(), + last_updated_ms: 1602638573590, + last_column_id: 3, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema1)), (1, Arc::new(schema2))]), + current_schema_id: 1, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), + default_spec_id: 0, + last_partition_id: 1000, + default_sort_order_id: 3, + sort_orders: HashMap::from_iter(vec![(3, sort_order.into())]), + snapshots: HashMap::from_iter(vec![ + (3051729675574597004, Arc::new(snapshot1)), + (3055729675574597004, Arc::new(snapshot2)), + ]), + current_snapshot_id: Some(3055729675574597004), + last_sequence_number: 34, + properties: HashMap::new(), + snapshot_log: vec![ + SnapshotLog { + snapshot_id: 3051729675574597004, + timestamp_ms: 1515100955770, + }, + SnapshotLog { + snapshot_id: 3055729675574597004, + timestamp_ms: 1555100955770, + }, + ], + metadata_log: Vec::new(), + refs: HashMap::from_iter(vec![( + "main".to_string(), + SnapshotReference { + snapshot_id: 3055729675574597004, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + }, + )]), + }; + + check_table_metadata_serde(&metadata, expected); + } + + #[test] + fn test_table_metadata_v2_file_valid_minimal() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2ValidMinimal.json").unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new( + NestedField::required(2, "y", Type::Primitive(PrimitiveType::Long)) + .with_doc("comment"), + ), + Arc::new(NestedField::required( + 3, + "z", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(0) + .with_partition_field(PartitionField { + name: "x".to_string(), + transform: Transform::Identity, + source_id: 1, + field_id: 1000, + }) + .build() + .unwrap(); + + let sort_order = SortOrder::builder() + .with_order_id(3) + .with_sort_field(SortField { + source_id: 2, + transform: Transform::Identity, + direction: SortDirection::Ascending, + null_order: NullOrder::First, + }) + .with_sort_field(SortField { + source_id: 3, + transform: Transform::Bucket(4), + direction: SortDirection::Descending, + null_order: NullOrder::Last, + }) + .build_unbound() + .unwrap(); + + let expected = TableMetadata { + format_version: FormatVersion::V2, + table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), + location: "s3://bucket/test/location".to_string(), + last_updated_ms: 1602638573590, + last_column_id: 3, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + current_schema_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), + default_spec_id: 0, + last_partition_id: 1000, + default_sort_order_id: 3, + sort_orders: HashMap::from_iter(vec![(3, sort_order.into())]), + snapshots: HashMap::default(), + current_snapshot_id: None, + last_sequence_number: 34, + properties: HashMap::new(), + snapshot_log: vec![], + metadata_log: Vec::new(), + refs: HashMap::new(), + }; + + check_table_metadata_serde(&metadata, expected); + } + + #[test] + fn test_table_metadata_v1_file_valid() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV1Valid.json").unwrap(); + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new( + NestedField::required(2, "y", Type::Primitive(PrimitiveType::Long)) + .with_doc("comment"), + ), + Arc::new(NestedField::required( + 3, + "z", + Type::Primitive(PrimitiveType::Long), + )), + ]) + .build() + .unwrap(); + + let partition_spec = PartitionSpec::builder() + .with_spec_id(0) + .with_partition_field(PartitionField { + name: "x".to_string(), + transform: Transform::Identity, + source_id: 1, + field_id: 1000, + }) + .build() + .unwrap(); + + let expected = TableMetadata { + format_version: FormatVersion::V1, + table_uuid: Uuid::parse_str("d20125c8-7284-442c-9aea-15fee620737c").unwrap(), + location: "s3://bucket/test/location".to_string(), + last_updated_ms: 1602638573874, + last_column_id: 3, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + current_schema_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), + default_spec_id: 0, + last_partition_id: 0, + default_sort_order_id: 0, + sort_orders: HashMap::new(), + snapshots: HashMap::new(), + current_snapshot_id: None, + last_sequence_number: 0, + properties: HashMap::new(), + snapshot_log: vec![], + metadata_log: Vec::new(), + refs: HashMap::from_iter(vec![( + "main".to_string(), + SnapshotReference { + snapshot_id: -1, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + }, + )]), + }; + + check_table_metadata_serde(&metadata, expected); + } + + #[test] + fn test_table_metadata_v2_schema_not_found() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json") + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "DataInvalid => No schema exists with the current schema id 2." + ) + } + + #[test] + fn test_table_metadata_v2_missing_sort_order() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2MissingSortOrder.json") + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "data did not match any variant of untagged enum TableMetadataEnum" + ) + } + + #[test] + fn test_table_metadata_v2_missing_partition_specs() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json") + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "data did not match any variant of untagged enum TableMetadataEnum" + ) + } + + #[test] + fn test_table_metadata_v2_missing_last_partition_id() { + let metadata = fs::read_to_string( + "testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json", + ) + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "data did not match any variant of untagged enum TableMetadataEnum" + ) + } + + #[test] + fn test_table_metadata_v2_missing_schemas() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataV2MissingSchemas.json") + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "data did not match any variant of untagged enum TableMetadataEnum" + ) + } + + #[test] + fn test_table_metadata_v2_unsupported_version() { + let metadata = + fs::read_to_string("testdata/table_metadata/TableMetadataUnsupportedVersion.json") + .unwrap(); + + let desered: Result = serde_json::from_str(&metadata); + + assert_eq!( + desered.unwrap_err().to_string(), + "data did not match any variant of untagged enum TableMetadataEnum" + ) + } + + #[test] + fn test_order_of_format_version() { + assert!(FormatVersion::V1 < FormatVersion::V2); + assert_eq!(FormatVersion::V1, FormatVersion::V1); + assert_eq!(FormatVersion::V2, FormatVersion::V2); + } + + #[test] + fn test_default_partition_spec() { + let default_spec_id = 1234; + let mut table_meta_data = get_test_table_metadata("TableMetadataV2Valid.json"); + table_meta_data.default_spec_id = default_spec_id; + table_meta_data + .partition_specs + .insert(default_spec_id, Arc::new(PartitionSpec::default())); + + assert_eq!( + table_meta_data.default_partition_spec(), + table_meta_data.partition_spec_by_id(default_spec_id) + ); + } + #[test] + fn test_default_sort_order() { + let default_sort_order_id = 1234; + let mut table_meta_data = get_test_table_metadata("TableMetadataV2Valid.json"); + table_meta_data.default_sort_order_id = default_sort_order_id; + table_meta_data + .sort_orders + .insert(default_sort_order_id, Arc::new(SortOrder::default())); + + assert_eq!( + table_meta_data.default_sort_order(), + table_meta_data.sort_orders.get(&default_sort_order_id) + ) + } +} diff --git a/crates/iceberg/src/spec/transform.rs b/crates/iceberg/src/spec/transform.rs new file mode 100644 index 0000000..839d582 --- /dev/null +++ b/crates/iceberg/src/spec/transform.rs @@ -0,0 +1,861 @@ +// 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. + +//! Transforms in iceberg. + +use crate::error::{Error, Result}; +use crate::spec::datatypes::{PrimitiveType, Type}; +use crate::ErrorKind; +use serde::{Deserialize, Deserializer, Serialize, Serializer}; +use std::fmt::{Display, Formatter}; +use std::str::FromStr; + +/// Transform is used to transform predicates to partition predicates, +/// in addition to transforming data values. +/// +/// Deriving partition predicates from column predicates on the table data +/// is used to separate the logical queries from physical storage: the +/// partitioning can change and the correct partition filters are always +/// derived from column predicates. +/// +/// This simplifies queries because users don’t have to supply both logical +/// predicates and partition predicates. +/// +/// All transforms must return `null` for a `null` input value. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum Transform { + /// Source value, unmodified + /// + /// - Source type could be any type. + /// - Return type is the same with source type. + Identity, + /// Hash of value, mod `N`. + /// + /// Bucket partition transforms use a 32-bit hash of the source value. + /// The 32-bit hash implementation is the 32-bit Murmur3 hash, x86 + /// variant, seeded with 0. + /// + /// Transforms are parameterized by a number of buckets, N. The hash mod + /// N must produce a positive value by first discarding the sign bit of + /// the hash value. In pseudo-code, the function is: + /// + /// ```text + /// def bucket_N(x) = (murmur3_x86_32_hash(x) & Integer.MAX_VALUE) % N + /// ``` + /// + /// - Source type could be `int`, `long`, `decimal`, `date`, `time`, + /// `timestamp`, `timestamptz`, `string`, `uuid`, `fixed`, `binary`. + /// - Return type is `int`. + Bucket(u32), + /// Value truncated to width `W` + /// + /// For `int`: + /// + /// - `v - (v % W)` remainders must be positive + /// - example: W=10: 1 → 0, -1 → -10 + /// - note: The remainder, v % W, must be positive. + /// + /// For `long`: + /// + /// - `v - (v % W)` remainders must be positive + /// - example: W=10: 1 → 0, -1 → -10 + /// - note: The remainder, v % W, must be positive. + /// + /// For `decimal`: + /// + /// - `scaled_W = decimal(W, scale(v)) v - (v % scaled_W)` + /// - example: W=50, s=2: 10.65 → 10.50 + /// + /// For `string`: + /// + /// - Substring of length L: `v.substring(0, L)` + /// - example: L=3: iceberg → ice + /// - note: Strings are truncated to a valid UTF-8 string with no more + /// than L code points. + /// + /// - Source type could be `int`, `long`, `decimal`, `string` + /// - Return type is the same with source type. + Truncate(u32), + /// Extract a date or timestamp year, as years from 1970 + /// + /// - Source type could be `date`, `timestamp`, `timestamptz` + /// - Return type is `int` + Year, + /// Extract a date or timestamp month, as months from 1970-01-01 + /// + /// - Source type could be `date`, `timestamp`, `timestamptz` + /// - Return type is `int` + Month, + /// Extract a date or timestamp day, as days from 1970-01-01 + /// + /// - Source type could be `date`, `timestamp`, `timestamptz` + /// - Return type is `int` + Day, + /// Extract a timestamp hour, as hours from 1970-01-01 00:00:00 + /// + /// - Source type could be `timestamp`, `timestamptz` + /// - Return type is `int` + Hour, + /// Always produces `null` + /// + /// The void transform may be used to replace the transform in an + /// existing partition field so that the field is effectively dropped in + /// v1 tables. + /// + /// - Source type could be any type.. + /// - Return type is Source type. + Void, + /// Used to represent some customized transform that can't be recognized or supported now. + Unknown, +} + +impl Transform { + /// Get the return type of transform given the input type. + /// Returns `None` if it can't be transformed. + pub fn result_type(&self, input_type: &Type) -> Result { + match self { + Transform::Identity => { + if matches!(input_type, Type::Primitive(_)) { + Ok(input_type.clone()) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of identity transform",), + )) + } + } + Transform::Void => Ok(input_type.clone()), + Transform::Unknown => Ok(Type::Primitive(PrimitiveType::String)), + Transform::Bucket(_) => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Int + | PrimitiveType::Long + | PrimitiveType::Decimal { .. } + | PrimitiveType::Date + | PrimitiveType::Time + | PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::String + | PrimitiveType::Uuid + | PrimitiveType::Fixed(_) + | PrimitiveType::Binary => Ok(Type::Primitive(PrimitiveType::Int)), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of bucket transform",), + )), + } + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of bucket transform",), + )) + } + } + Transform::Truncate(_) => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Int + | PrimitiveType::Long + | PrimitiveType::String + | PrimitiveType::Binary + | PrimitiveType::Decimal { .. } => Ok(input_type.clone()), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of truncate transform",), + )), + } + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of truncate transform",), + )) + } + } + Transform::Year | Transform::Month | Transform::Day => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::Date => Ok(Type::Primitive(PrimitiveType::Int)), + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of {self} transform",), + )), + } + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of {self} transform",), + )) + } + } + Transform::Hour => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Timestamp | PrimitiveType::Timestamptz => { + Ok(Type::Primitive(PrimitiveType::Int)) + } + _ => Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of {self} transform",), + )), + } + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("{input_type} is not a valid input type of {self} transform",), + )) + } + } + } + } + + /// Whether the transform preserves the order of values. + pub fn preserves_order(&self) -> bool { + !matches!( + self, + Transform::Void | Transform::Bucket(_) | Transform::Unknown + ) + } + + /// Return the unique transform name to check if similar transforms for the same source field + /// are added multiple times in partition spec builder. + pub fn dedup_name(&self) -> String { + match self { + Transform::Year | Transform::Month | Transform::Day | Transform::Hour => { + "time".to_string() + } + _ => format!("{self}"), + } + } + + /// Whether ordering by this transform's result satisfies the ordering of another transform's + /// result. + /// + /// For example, sorting by day(ts) will produce an ordering that is also by month(ts) or + // year(ts). However, sorting by day(ts) will not satisfy the order of hour(ts) or identity(ts). + pub fn satisfies_order_of(&self, other: &Self) -> bool { + match self { + Transform::Identity => other.preserves_order(), + Transform::Hour => matches!( + other, + Transform::Hour | Transform::Day | Transform::Month | Transform::Year + ), + Transform::Day => matches!(other, Transform::Day | Transform::Month | Transform::Year), + Transform::Month => matches!(other, Transform::Month | Transform::Year), + _ => self == other, + } + } +} + +impl Display for Transform { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Transform::Identity => write!(f, "identity"), + Transform::Year => write!(f, "year"), + Transform::Month => write!(f, "month"), + Transform::Day => write!(f, "day"), + Transform::Hour => write!(f, "hour"), + Transform::Void => write!(f, "void"), + Transform::Bucket(length) => write!(f, "bucket[{length}]"), + Transform::Truncate(width) => write!(f, "truncate[{width}]"), + Transform::Unknown => write!(f, "unknown"), + } + } +} + +impl FromStr for Transform { + type Err = Error; + + fn from_str(s: &str) -> Result { + let t = match s { + "identity" => Transform::Identity, + "year" => Transform::Year, + "month" => Transform::Month, + "day" => Transform::Day, + "hour" => Transform::Hour, + "void" => Transform::Void, + "unknown" => Transform::Unknown, + v if v.starts_with("bucket") => { + let length = v + .strip_prefix("bucket") + .expect("transform must starts with `bucket`") + .trim_start_matches('[') + .trim_end_matches(']') + .parse() + .map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + format!("transform bucket type {v:?} is invalid"), + ) + .with_source(err) + })?; + + Transform::Bucket(length) + } + v if v.starts_with("truncate") => { + let width = v + .strip_prefix("truncate") + .expect("transform must starts with `truncate`") + .trim_start_matches('[') + .trim_end_matches(']') + .parse() + .map_err(|err| { + Error::new( + ErrorKind::DataInvalid, + format!("transform truncate type {v:?} is invalid"), + ) + .with_source(err) + })?; + + Transform::Truncate(width) + } + v => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("transform {v:?} is invalid"), + )) + } + }; + + Ok(t) + } +} + +impl Serialize for Transform { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: Serializer, + { + serializer.serialize_str(format!("{self}").as_str()) + } +} + +impl<'de> Deserialize<'de> for Transform { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + s.parse().map_err(::custom) + } +} + +#[cfg(test)] +mod tests { + use crate::spec::datatypes::PrimitiveType::{ + Binary, Date, Decimal, Fixed, Int, Long, String as StringType, Time, Timestamp, + Timestamptz, Uuid, + }; + use crate::spec::datatypes::Type::{Primitive, Struct}; + use crate::spec::datatypes::{NestedField, StructType, Type}; + use crate::spec::transform::Transform; + + struct TestParameter { + display: String, + json: String, + dedup_name: String, + preserves_order: bool, + satisfies_order_of: Vec<(Transform, bool)>, + trans_types: Vec<(Type, Option)>, + } + + fn check_transform(trans: Transform, param: TestParameter) { + assert_eq!(param.display, format!("{trans}")); + assert_eq!(param.json, serde_json::to_string(&trans).unwrap()); + assert_eq!(trans, serde_json::from_str(param.json.as_str()).unwrap()); + assert_eq!(param.dedup_name, trans.dedup_name()); + assert_eq!(param.preserves_order, trans.preserves_order()); + + for (other_trans, satisfies_order_of) in param.satisfies_order_of { + assert_eq!( + satisfies_order_of, + trans.satisfies_order_of(&other_trans), + "Failed to check satisfies order {}, {}, {}", + trans, + other_trans, + satisfies_order_of + ); + } + + for (input_type, result_type) in param.trans_types { + assert_eq!(result_type, trans.result_type(&input_type).ok()); + } + } + + #[test] + fn test_bucket_transform() { + let trans = Transform::Bucket(8); + + let test_param = TestParameter { + display: "bucket[8]".to_string(), + json: r#""bucket[8]""#.to_string(), + dedup_name: "bucket[8]".to_string(), + preserves_order: false, + satisfies_order_of: vec![ + (Transform::Bucket(8), true), + (Transform::Bucket(4), false), + (Transform::Void, false), + (Transform::Day, false), + ], + trans_types: vec![ + (Primitive(Binary), Some(Primitive(Int))), + (Primitive(Date), Some(Primitive(Int))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + Some(Primitive(Int)), + ), + (Primitive(Fixed(8)), Some(Primitive(Int))), + (Primitive(Int), Some(Primitive(Int))), + (Primitive(Long), Some(Primitive(Int))), + (Primitive(StringType), Some(Primitive(Int))), + (Primitive(Uuid), Some(Primitive(Int))), + (Primitive(Time), Some(Primitive(Int))), + (Primitive(Timestamp), Some(Primitive(Int))), + (Primitive(Timestamptz), Some(Primitive(Int))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_truncate_transform() { + let trans = Transform::Truncate(4); + + let test_param = TestParameter { + display: "truncate[4]".to_string(), + json: r#""truncate[4]""#.to_string(), + dedup_name: "truncate[4]".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Truncate(4), true), + (Transform::Truncate(2), false), + (Transform::Bucket(4), false), + (Transform::Void, false), + (Transform::Day, false), + ], + trans_types: vec![ + (Primitive(Binary), Some(Primitive(Binary))), + (Primitive(Date), None), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + Some(Primitive(Decimal { + precision: 8, + scale: 5, + })), + ), + (Primitive(Fixed(8)), None), + (Primitive(Int), Some(Primitive(Int))), + (Primitive(Long), Some(Primitive(Long))), + (Primitive(StringType), Some(Primitive(StringType))), + (Primitive(Uuid), None), + (Primitive(Time), None), + (Primitive(Timestamp), None), + (Primitive(Timestamptz), None), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_identity_transform() { + let trans = Transform::Identity; + + let test_param = TestParameter { + display: "identity".to_string(), + json: r#""identity""#.to_string(), + dedup_name: "identity".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Truncate(4), true), + (Transform::Truncate(2), true), + (Transform::Bucket(4), false), + (Transform::Void, false), + (Transform::Day, true), + ], + trans_types: vec![ + (Primitive(Binary), Some(Primitive(Binary))), + (Primitive(Date), Some(Primitive(Date))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + Some(Primitive(Decimal { + precision: 8, + scale: 5, + })), + ), + (Primitive(Fixed(8)), Some(Primitive(Fixed(8)))), + (Primitive(Int), Some(Primitive(Int))), + (Primitive(Long), Some(Primitive(Long))), + (Primitive(StringType), Some(Primitive(StringType))), + (Primitive(Uuid), Some(Primitive(Uuid))), + (Primitive(Time), Some(Primitive(Time))), + (Primitive(Timestamp), Some(Primitive(Timestamp))), + (Primitive(Timestamptz), Some(Primitive(Timestamptz))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_year_transform() { + let trans = Transform::Year; + + let test_param = TestParameter { + display: "year".to_string(), + json: r#""year""#.to_string(), + dedup_name: "time".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Year, true), + (Transform::Month, false), + (Transform::Day, false), + (Transform::Hour, false), + (Transform::Void, false), + (Transform::Identity, false), + ], + trans_types: vec![ + (Primitive(Binary), None), + (Primitive(Date), Some(Primitive(Int))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + None, + ), + (Primitive(Fixed(8)), None), + (Primitive(Int), None), + (Primitive(Long), None), + (Primitive(StringType), None), + (Primitive(Uuid), None), + (Primitive(Time), None), + (Primitive(Timestamp), Some(Primitive(Int))), + (Primitive(Timestamptz), Some(Primitive(Int))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_month_transform() { + let trans = Transform::Month; + + let test_param = TestParameter { + display: "month".to_string(), + json: r#""month""#.to_string(), + dedup_name: "time".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Year, true), + (Transform::Month, true), + (Transform::Day, false), + (Transform::Hour, false), + (Transform::Void, false), + (Transform::Identity, false), + ], + trans_types: vec![ + (Primitive(Binary), None), + (Primitive(Date), Some(Primitive(Int))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + None, + ), + (Primitive(Fixed(8)), None), + (Primitive(Int), None), + (Primitive(Long), None), + (Primitive(StringType), None), + (Primitive(Uuid), None), + (Primitive(Time), None), + (Primitive(Timestamp), Some(Primitive(Int))), + (Primitive(Timestamptz), Some(Primitive(Int))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_day_transform() { + let trans = Transform::Day; + + let test_param = TestParameter { + display: "day".to_string(), + json: r#""day""#.to_string(), + dedup_name: "time".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Year, true), + (Transform::Month, true), + (Transform::Day, true), + (Transform::Hour, false), + (Transform::Void, false), + (Transform::Identity, false), + ], + trans_types: vec![ + (Primitive(Binary), None), + (Primitive(Date), Some(Primitive(Int))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + None, + ), + (Primitive(Fixed(8)), None), + (Primitive(Int), None), + (Primitive(Long), None), + (Primitive(StringType), None), + (Primitive(Uuid), None), + (Primitive(Time), None), + (Primitive(Timestamp), Some(Primitive(Int))), + (Primitive(Timestamptz), Some(Primitive(Int))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_hour_transform() { + let trans = Transform::Hour; + + let test_param = TestParameter { + display: "hour".to_string(), + json: r#""hour""#.to_string(), + dedup_name: "time".to_string(), + preserves_order: true, + satisfies_order_of: vec![ + (Transform::Year, true), + (Transform::Month, true), + (Transform::Day, true), + (Transform::Hour, true), + (Transform::Void, false), + (Transform::Identity, false), + ], + trans_types: vec![ + (Primitive(Binary), None), + (Primitive(Date), None), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + None, + ), + (Primitive(Fixed(8)), None), + (Primitive(Int), None), + (Primitive(Long), None), + (Primitive(StringType), None), + (Primitive(Uuid), None), + (Primitive(Time), None), + (Primitive(Timestamp), Some(Primitive(Int))), + (Primitive(Timestamptz), Some(Primitive(Int))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + None, + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_void_transform() { + let trans = Transform::Void; + + let test_param = TestParameter { + display: "void".to_string(), + json: r#""void""#.to_string(), + dedup_name: "void".to_string(), + preserves_order: false, + satisfies_order_of: vec![ + (Transform::Year, false), + (Transform::Month, false), + (Transform::Day, false), + (Transform::Hour, false), + (Transform::Void, true), + (Transform::Identity, false), + ], + trans_types: vec![ + (Primitive(Binary), Some(Primitive(Binary))), + (Primitive(Date), Some(Primitive(Date))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + Some(Primitive(Decimal { + precision: 8, + scale: 5, + })), + ), + (Primitive(Fixed(8)), Some(Primitive(Fixed(8)))), + (Primitive(Int), Some(Primitive(Int))), + (Primitive(Long), Some(Primitive(Long))), + (Primitive(StringType), Some(Primitive(StringType))), + (Primitive(Uuid), Some(Primitive(Uuid))), + (Primitive(Time), Some(Primitive(Time))), + (Primitive(Timestamp), Some(Primitive(Timestamp))), + (Primitive(Timestamptz), Some(Primitive(Timestamptz))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + Some(Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()]))), + ), + ], + }; + + check_transform(trans, test_param); + } + + #[test] + fn test_known_transform() { + let trans = Transform::Unknown; + + let test_param = TestParameter { + display: "unknown".to_string(), + json: r#""unknown""#.to_string(), + dedup_name: "unknown".to_string(), + preserves_order: false, + satisfies_order_of: vec![ + (Transform::Year, false), + (Transform::Month, false), + (Transform::Day, false), + (Transform::Hour, false), + (Transform::Void, false), + (Transform::Identity, false), + (Transform::Unknown, true), + ], + trans_types: vec![ + (Primitive(Binary), Some(Primitive(StringType))), + (Primitive(Date), Some(Primitive(StringType))), + ( + Primitive(Decimal { + precision: 8, + scale: 5, + }), + Some(Primitive(StringType)), + ), + (Primitive(Fixed(8)), Some(Primitive(StringType))), + (Primitive(Int), Some(Primitive(StringType))), + (Primitive(Long), Some(Primitive(StringType))), + (Primitive(StringType), Some(Primitive(StringType))), + (Primitive(Uuid), Some(Primitive(StringType))), + (Primitive(Time), Some(Primitive(StringType))), + (Primitive(Timestamp), Some(Primitive(StringType))), + (Primitive(Timestamptz), Some(Primitive(StringType))), + ( + Struct(StructType::new(vec![NestedField::optional( + 1, + "a", + Primitive(Timestamp), + ) + .into()])), + Some(Primitive(StringType)), + ), + ], + }; + + check_transform(trans, test_param); + } +} diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs new file mode 100644 index 0000000..a8a748d --- /dev/null +++ b/crates/iceberg/src/spec/values.rs @@ -0,0 +1,2237 @@ +// 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. + +/*! + * Value in iceberg + */ + +use std::str::FromStr; +use std::{any::Any, collections::BTreeMap}; + +use crate::error::Result; +use bitvec::vec::BitVec; +use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; +use ordered_float::OrderedFloat; +use rust_decimal::Decimal; +use serde_bytes::ByteBuf; +use serde_json::{Map as JsonMap, Number, Value as JsonValue}; +use uuid::Uuid; + +use crate::{Error, ErrorKind}; + +use super::datatypes::{PrimitiveType, Type}; + +pub use _serde::RawLiteral; + +/// Values present in iceberg type +#[derive(Clone, Debug, PartialEq, Hash, Eq, PartialOrd, Ord)] +pub enum PrimitiveLiteral { + /// 0x00 for false, non-zero byte for true + Boolean(bool), + /// Stored as 4-byte little-endian + Int(i32), + /// Stored as 8-byte little-endian + Long(i64), + /// Stored as 4-byte little-endian + Float(OrderedFloat), + /// Stored as 8-byte little-endian + Double(OrderedFloat), + /// Stores days from the 1970-01-01 in an 4-byte little-endian int + Date(i32), + /// Stores microseconds from midnight in an 8-byte little-endian long + Time(i64), + /// Timestamp without timezone + Timestamp(i64), + /// Timestamp with timezone + TimestampTZ(i64), + /// UTF-8 bytes (without length) + String(String), + /// 16-byte big-endian value + UUID(Uuid), + /// Binary value + Fixed(Vec), + /// Binary value (without length) + Binary(Vec), + /// Stores unscaled value as big int. According to iceberg spec, the precision must less than 38(`MAX_DECIMAL_PRECISION`) , so i128 is suit here. + Decimal(i128), +} + +/// Values present in iceberg type +#[derive(Clone, Debug, PartialEq, Hash, Eq, PartialOrd, Ord)] +pub enum Literal { + /// A primitive value + Primitive(PrimitiveLiteral), + /// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. + /// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. + /// Fields may have an optional comment or doc string. Fields can have default values. + Struct(Struct), + /// A list is a collection of values with some element type. + /// The element field has an integer id that is unique in the table schema. + /// Elements can be either optional or required. Element types may be any type. + List(Vec>), + /// A map is a collection of key-value pairs with a key type and a value type. + /// Both the key field and value field each have an integer id that is unique in the table schema. + /// Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. + Map(BTreeMap>), +} + +impl Literal { + /// Creates a boolean value. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool(true); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(true)), t); + /// ``` + pub fn bool>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Boolean(t.into())) + } + + /// Creates a boolean value from string. + /// See [Parse bool from str](https://doc.rust-lang.org/stable/std/primitive.bool.html#impl-FromStr-for-bool) for reference. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::bool_from_str("false").unwrap(); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Boolean(false)), t); + /// ``` + pub fn bool_from_str>(s: S) -> Result { + let v = s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to bool.").with_source(e) + })?; + Ok(Self::Primitive(PrimitiveLiteral::Boolean(v))) + } + + /// Creates an 32bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::int(23i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Int(23)), t); + /// ``` + pub fn int>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Int(t.into())) + } + + /// Creates an 64bit integer. + /// + /// Example: + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::long(24i8); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Long(24)), t); + /// ``` + pub fn long>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Long(t.into())) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::float( 32.1f32 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(32.1))), t); + /// ``` + pub fn float>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Float(OrderedFloat(t.into()))) + } + + /// Creates an 32bit floating point number. + /// + /// Example: + /// ```rust + /// use ordered_float::OrderedFloat; + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t = Literal::double( 32.1f64 ); + /// + /// assert_eq!(Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(32.1))), t); + /// ``` + pub fn double>(t: T) -> Self { + Self::Primitive(PrimitiveLiteral::Double(OrderedFloat(t.into()))) + } + + /// Returns unix epoch. + pub fn unix_epoch() -> DateTime { + Utc.timestamp_nanos(0) + } + + /// Creates date literal from number of days from unix epoch directly. + pub fn date(days: i32) -> Self { + Self::Primitive(PrimitiveLiteral::Date(days)) + } + + /// Creates date literal from `NaiveDate`, assuming it's utc timezone. + fn date_from_naive_date(date: NaiveDate) -> Self { + let days = (date - Self::unix_epoch().date_naive()).num_days(); + Self::date(days as i32) + } + + /// Creates a date in `%Y-%m-%d` format, assume in utc timezone. + /// + /// See [`NaiveDate::from_str`]. + /// + /// Example + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_str("1970-01-03").unwrap(); + /// + /// assert_eq!(Literal::date(2), t); + /// ``` + pub fn date_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse date from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::date_from_naive_date(t)) + } + + /// Create a date from calendar date (year, month and day). + /// + /// See [`NaiveDate::from_ymd_opt`]. + /// + /// Example: + /// + ///```rust + /// use iceberg::spec::Literal; + /// let t = Literal::date_from_ymd(1970, 1, 5).unwrap(); + /// + /// assert_eq!(Literal::date(4), t); + /// ``` + pub fn date_from_ymd(year: i32, month: u32, day: u32) -> Result { + let t = NaiveDate::from_ymd_opt(year, month, day).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't create date from year: {year}, month: {month}, day: {day}"), + ) + })?; + + Ok(Self::date_from_naive_date(t)) + } + + /// Creates time in microseconds directly + pub fn time(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Time(value)) + } + + /// Creates time literal from [`chrono::NaiveTime`]. + fn time_from_naive_time(t: NaiveTime) -> Self { + let duration = t - Self::unix_epoch().time(); + // It's safe to unwrap here since less than 24 hours will never overflow. + let micro_secs = duration.num_microseconds().unwrap(); + + Literal::time(micro_secs) + } + + /// Creates time in microseconds in `%H:%M:%S:.f` format. + /// + /// See [`NaiveTime::from_str`] for details. + /// + /// Example: + /// ```rust + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_str("01:02:01.888999777").unwrap(); + /// + /// let micro_secs = { + /// 1 * 3600 * 1_000_000 + // 1 hour + /// 2 * 60 * 1_000_000 + // 2 minutes + /// 1 * 1_000_000 + // 1 second + /// 888999 // microseconds + /// }; + /// assert_eq!(Literal::time(micro_secs), t); + /// ``` + pub fn time_from_str>(s: S) -> Result { + let t = s.as_ref().parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse time from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + + Ok(Self::time_from_naive_time(t)) + } + + /// Creates time literal from hour, minute, second, and microseconds. + /// + /// See [`NaiveTime::from_hms_micro_opt`]. + /// + /// Example: + /// ```rust + /// + /// use iceberg::spec::Literal; + /// let t = Literal::time_from_hms_micro(22, 15, 33, 111).unwrap(); + /// + /// assert_eq!(Literal::time_from_str("22:15:33.000111").unwrap(), t); + /// ``` + pub fn time_from_hms_micro(hour: u32, min: u32, sec: u32, micro: u32) -> Result { + let t = NaiveTime::from_hms_micro_opt(hour, min, sec, micro) + .ok_or_else(|| Error::new( + ErrorKind::DataInvalid, + format!("Can't create time from hour: {hour}, min: {min}, second: {sec}, microsecond: {micro}"), + ))?; + Ok(Self::time_from_naive_time(t)) + } + + /// Creates a timestamp from unix epoch in microseconds. + pub fn timestamp(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::Timestamp(value)) + } + + /// Creates a timestamp with timezone from unix epoch in microseconds. + pub fn timestamptz(value: i64) -> Self { + Self::Primitive(PrimitiveLiteral::TimestampTZ(value)) + } + + /// Creates a timestamp from [`DateTime`]. + pub fn timestamp_from_datetime(dt: DateTime) -> Self { + Self::timestamp(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Creates a timestamp with timezone from [`DateTime`]. + pub fn timestamptz_from_datetime(dt: DateTime) -> Self { + Self::timestamptz(dt.with_timezone(&Utc).timestamp_micros()) + } + + /// Parse a timestamp in RFC3339 format. + /// + /// See [`DateTime::from_str`]. + /// + /// Example: + /// + /// ```rust + /// use chrono::{DateTime, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime}; + /// use iceberg::spec::Literal; + /// let t = Literal::timestamp_from_str("2012-12-12 12:12:12.8899-04:00").unwrap(); + /// + /// let t2 = { + /// let date = NaiveDate::from_ymd_opt(2012, 12, 12).unwrap(); + /// let time = NaiveTime::from_hms_micro_opt(12, 12, 12, 889900).unwrap(); + /// let dt = NaiveDateTime::new(date, time); + /// Literal::timestamp_from_datetime(DateTime::::from_local(dt, FixedOffset::west_opt(4 * 3600).unwrap())) + /// }; + /// + /// assert_eq!(t, t2); + /// ``` + pub fn timestamp_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamp_from_datetime(dt)) + } + + /// Similar to [`Literal::timestamp_from_str`], but return timestamp with timezone literal. + pub fn timestamptz_from_str>(s: S) -> Result { + let dt = DateTime::::from_str(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse datetime.").with_source(e) + })?; + + Ok(Self::timestamptz_from_datetime(dt)) + } + + /// Creates a string literal. + pub fn string(s: S) -> Self { + Self::Primitive(PrimitiveLiteral::String(s.to_string())) + } + + /// Creates uuid literal. + pub fn uuid(uuid: Uuid) -> Self { + Self::Primitive(PrimitiveLiteral::UUID(uuid)) + } + + /// Creates uuid from str. See [`Uuid::parse_str`]. + /// + /// Example: + /// + /// ```rust + /// use uuid::Uuid; + /// use iceberg::spec::Literal; + /// let t1 = Literal::uuid_from_str("a1a2a3a4-b1b2-c1c2-d1d2-d3d4d5d6d7d8").unwrap(); + /// let t2 = Literal::uuid(Uuid::from_u128_le(0xd8d7d6d5d4d3d2d1c2c1b2b1a4a3a2a1)); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn uuid_from_str>(s: S) -> Result { + let uuid = Uuid::parse_str(s.as_ref()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't parse uuid from string: {}", s.as_ref()), + ) + .with_source(e) + })?; + Ok(Self::uuid(uuid)) + } + + /// Creates a fixed literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::fixed(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Fixed(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn fixed>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Fixed(input.into_iter().collect())) + } + + /// Creates a binary literal from bytes. + /// + /// Example: + /// + /// ```rust + /// use iceberg::spec::{Literal, PrimitiveLiteral}; + /// let t1 = Literal::binary(vec![1u8, 2u8]); + /// let t2 = Literal::Primitive(PrimitiveLiteral::Binary(vec![1u8, 2u8])); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn binary>(input: I) -> Self { + Literal::Primitive(PrimitiveLiteral::Binary(input.into_iter().collect())) + } + + /// Creates a decimal literal. + pub fn decimal(decimal: i128) -> Self { + Self::Primitive(PrimitiveLiteral::Decimal(decimal)) + } + + /// Creates decimal literal from string. See [`Decimal::from_str_exact`]. + /// + /// Example: + /// + /// ```rust + /// use rust_decimal::Decimal; + /// use iceberg::spec::Literal; + /// let t1 = Literal::decimal(12345); + /// let t2 = Literal::decimal_from_str("123.45").unwrap(); + /// + /// assert_eq!(t1, t2); + /// ``` + pub fn decimal_from_str>(s: S) -> Result { + let decimal = Decimal::from_str_exact(s.as_ref()).map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse decimal.").with_source(e) + })?; + Ok(Self::decimal(decimal.mantissa())) + } +} + +impl From for ByteBuf { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(val) => { + if val { + ByteBuf::from([1u8]) + } else { + ByteBuf::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::TimestampTZ(val) => ByteBuf::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => ByteBuf::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => ByteBuf::from(val), + PrimitiveLiteral::Binary(val) => ByteBuf::from(val), + PrimitiveLiteral::Decimal(_) => todo!(), + }, + _ => unimplemented!(), + } + } +} + +impl From for Vec { + fn from(value: Literal) -> Self { + match value { + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(val) => { + if val { + Vec::from([1u8]) + } else { + Vec::from([0u8]) + } + } + PrimitiveLiteral::Int(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Long(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Float(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Double(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Date(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Time(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::Timestamp(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::TimestampTZ(val) => Vec::from(val.to_le_bytes()), + PrimitiveLiteral::String(val) => Vec::from(val.as_bytes()), + PrimitiveLiteral::UUID(val) => Vec::from(val.as_u128().to_be_bytes()), + PrimitiveLiteral::Fixed(val) => val, + PrimitiveLiteral::Binary(val) => val, + PrimitiveLiteral::Decimal(_) => todo!(), + }, + _ => unimplemented!(), + } + } +} + +/// The partition struct stores the tuple of partition values for each file. +/// Its type is derived from the partition fields of the partition spec used to write the manifest file. +/// In v2, the partition struct’s field ids must match the ids from the partition spec. +#[derive(Debug, Clone, PartialEq, Hash, Eq, PartialOrd, Ord)] +pub struct Struct { + /// Vector to store the field values + fields: Vec, + /// Null bitmap + null_bitmap: BitVec, +} + +impl Struct { + /// Create a empty struct. + pub fn empty() -> Self { + Self { + fields: Vec::new(), + null_bitmap: BitVec::new(), + } + } + + /// Create a iterator to read the field in order of field_value. + pub fn iter(&self) -> impl Iterator> { + self.null_bitmap.iter().zip(self.fields.iter()).map( + |(null, value)| { + if *null { + None + } else { + Some(value) + } + }, + ) + } +} + +/// An iterator that moves out of a struct. +pub struct StructValueIntoIter { + null_bitmap: bitvec::boxed::IntoIter, + fields: std::vec::IntoIter, +} + +impl Iterator for StructValueIntoIter { + type Item = Option; + + fn next(&mut self) -> Option { + match (self.null_bitmap.next(), self.fields.next()) { + (Some(null), Some(value)) => Some(if null { None } else { Some(value) }), + _ => None, + } + } +} + +impl IntoIterator for Struct { + type Item = Option; + + type IntoIter = StructValueIntoIter; + + fn into_iter(self) -> Self::IntoIter { + StructValueIntoIter { + null_bitmap: self.null_bitmap.into_iter(), + fields: self.fields.into_iter(), + } + } +} + +impl FromIterator> for Struct { + fn from_iter>>(iter: I) -> Self { + let mut fields = Vec::new(); + let mut null_bitmap = BitVec::new(); + + for value in iter.into_iter() { + match value { + Some(value) => { + fields.push(value); + null_bitmap.push(false) + } + None => { + fields.push(Literal::Primitive(PrimitiveLiteral::Boolean(false))); + null_bitmap.push(true) + } + } + } + Struct { + fields, + null_bitmap, + } + } +} + +impl Literal { + /// Create iceberg value from bytes + pub fn try_from_bytes(bytes: &[u8], data_type: &Type) -> Result { + match data_type { + Type::Primitive(primitive) => match primitive { + PrimitiveType::Boolean => { + if bytes.len() == 1 && bytes[0] == 0u8 { + Ok(Literal::Primitive(PrimitiveLiteral::Boolean(false))) + } else { + Ok(Literal::Primitive(PrimitiveLiteral::Boolean(true))) + } + } + PrimitiveType::Int => Ok(Literal::Primitive(PrimitiveLiteral::Int( + i32::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Long => Ok(Literal::Primitive(PrimitiveLiteral::Long( + i64::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Float => Ok(Literal::Primitive(PrimitiveLiteral::Float( + OrderedFloat(f32::from_le_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Double => Ok(Literal::Primitive(PrimitiveLiteral::Double( + OrderedFloat(f64::from_le_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Date => Ok(Literal::Primitive(PrimitiveLiteral::Date( + i32::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Time => Ok(Literal::Primitive(PrimitiveLiteral::Time( + i64::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Timestamp => Ok(Literal::Primitive(PrimitiveLiteral::Timestamp( + i64::from_le_bytes(bytes.try_into()?), + ))), + PrimitiveType::Timestamptz => Ok(Literal::Primitive( + PrimitiveLiteral::TimestampTZ(i64::from_le_bytes(bytes.try_into()?)), + )), + PrimitiveType::String => Ok(Literal::Primitive(PrimitiveLiteral::String( + std::str::from_utf8(bytes)?.to_string(), + ))), + PrimitiveType::Uuid => Ok(Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::from_u128(u128::from_be_bytes(bytes.try_into()?)), + ))), + PrimitiveType::Fixed(_) => Ok(Literal::Primitive(PrimitiveLiteral::Fixed( + Vec::from(bytes), + ))), + PrimitiveType::Binary => Ok(Literal::Primitive(PrimitiveLiteral::Binary( + Vec::from(bytes), + ))), + PrimitiveType::Decimal { + precision: _, + scale: _, + } => todo!(), + }, + _ => Err(Error::new( + crate::ErrorKind::DataInvalid, + "Converting bytes to non-primitive types is not supported.", + )), + } + } + + /// Create iceberg value from a json value + pub fn try_from_json(value: JsonValue, data_type: &Type) -> Result> { + match data_type { + Type::Primitive(primitive) => match (primitive, value) { + (PrimitiveType::Boolean, JsonValue::Bool(bool)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Boolean(bool)))) + } + (PrimitiveType::Int, JsonValue::Number(number)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Int( + number + .as_i64() + .ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to int", + ))? + .try_into()?, + )))) + } + (PrimitiveType::Long, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Long(number.as_i64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to long", + ))?), + ))), + (PrimitiveType::Float, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Float(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to float", + ))? as f32)), + ))), + (PrimitiveType::Double, JsonValue::Number(number)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Double(OrderedFloat(number.as_f64().ok_or(Error::new( + crate::ErrorKind::DataInvalid, + "Failed to convert json number to double", + ))?)), + ))), + (PrimitiveType::Date, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Date( + date::date_to_days(&NaiveDate::parse_from_str(&s, "%Y-%m-%d")?), + )))) + } + (PrimitiveType::Time, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::Time( + time::time_to_microseconds(&NaiveTime::parse_from_str(&s, "%H:%M:%S%.f")?), + )))) + } + (PrimitiveType::Timestamp, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::Timestamp(timestamp::datetime_to_microseconds( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f")?, + )), + ))), + (PrimitiveType::Timestamptz, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::TimestampTZ( + timestamptz::datetimetz_to_microseconds(&Utc.from_utc_datetime( + &NaiveDateTime::parse_from_str(&s, "%Y-%m-%dT%H:%M:%S%.f+00:00")?, + )), + )))) + } + (PrimitiveType::String, JsonValue::String(s)) => { + Ok(Some(Literal::Primitive(PrimitiveLiteral::String(s)))) + } + (PrimitiveType::Uuid, JsonValue::String(s)) => Ok(Some(Literal::Primitive( + PrimitiveLiteral::UUID(Uuid::parse_str(&s)?), + ))), + (PrimitiveType::Fixed(_), JsonValue::String(_)) => todo!(), + (PrimitiveType::Binary, JsonValue::String(_)) => todo!(), + ( + PrimitiveType::Decimal { + precision: _, + scale, + }, + JsonValue::String(s), + ) => { + let mut decimal = Decimal::from_str_exact(&s)?; + decimal.rescale(*scale); + Ok(Some(Literal::Primitive(PrimitiveLiteral::Decimal( + decimal.mantissa(), + )))) + } + (_, JsonValue::Null) => Ok(None), + (i, j) => Err(Error::new( + crate::ErrorKind::DataInvalid, + format!( + "The json value {} doesn't fit to the iceberg type {}.", + j, i + ), + )), + }, + Type::Struct(schema) => { + if let JsonValue::Object(mut object) = value { + Ok(Some(Literal::Struct(Struct::from_iter( + schema.fields().iter().map(|field| { + object.remove(&field.id.to_string()).and_then(|value| { + Literal::try_from_json(value, &field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + }) + .ok() + }) + }), + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a struct type must be an object.", + )) + } + } + Type::List(list) => { + if let JsonValue::Array(array) = value { + Ok(Some(Literal::List( + array + .into_iter() + .map(|value| { + Literal::try_from_json(value, &list.element_field.field_type) + }) + .collect::>>()?, + ))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + Type::Map(map) => { + if let JsonValue::Object(mut object) = value { + if let (Some(JsonValue::Array(keys)), Some(JsonValue::Array(values))) = + (object.remove("keys"), object.remove("values")) + { + Ok(Some(Literal::Map(BTreeMap::from_iter( + keys.into_iter() + .zip(values.into_iter()) + .map(|(key, value)| { + Ok(( + Literal::try_from_json(key, &map.key_field.field_type) + .and_then(|value| { + value.ok_or(Error::new( + ErrorKind::DataInvalid, + "Key of map cannot be null", + )) + })?, + Literal::try_from_json(value, &map.value_field.field_type)?, + )) + }) + .collect::>>()?, + )))) + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } else { + Err(Error::new( + crate::ErrorKind::DataInvalid, + "The json value for a list type must be an array.", + )) + } + } + } + } + + /// Converting iceberg value to json value. + /// + /// See [this spec](https://iceberg.apache.org/spec/#json-single-value-serialization) for reference. + pub fn try_into_json(self, r#type: &Type) -> Result { + match (self, r#type) { + (Literal::Primitive(prim), _) => match prim { + PrimitiveLiteral::Boolean(val) => Ok(JsonValue::Bool(val)), + PrimitiveLiteral::Int(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Long(val) => Ok(JsonValue::Number((val).into())), + PrimitiveLiteral::Float(val) => match Number::from_f64(val.0 as f64) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Double(val) => match Number::from_f64(val.0) { + Some(number) => Ok(JsonValue::Number(number)), + None => Ok(JsonValue::Null), + }, + PrimitiveLiteral::Date(val) => { + Ok(JsonValue::String(date::days_to_date(val).to_string())) + } + PrimitiveLiteral::Time(val) => Ok(JsonValue::String( + time::microseconds_to_time(val).to_string(), + )), + PrimitiveLiteral::Timestamp(val) => Ok(JsonValue::String( + timestamp::microseconds_to_datetime(val) + .format("%Y-%m-%dT%H:%M:%S%.f") + .to_string(), + )), + PrimitiveLiteral::TimestampTZ(val) => Ok(JsonValue::String( + timestamptz::microseconds_to_datetimetz(val) + .format("%Y-%m-%dT%H:%M:%S%.f+00:00") + .to_string(), + )), + PrimitiveLiteral::String(val) => Ok(JsonValue::String(val.clone())), + PrimitiveLiteral::UUID(val) => Ok(JsonValue::String(val.to_string())), + PrimitiveLiteral::Fixed(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Binary(val) => Ok(JsonValue::String(val.iter().fold( + String::new(), + |mut acc, x| { + acc.push_str(&format!("{:x}", x)); + acc + }, + ))), + PrimitiveLiteral::Decimal(val) => match r#type { + Type::Primitive(PrimitiveType::Decimal { + precision: _precision, + scale, + }) => { + let decimal = Decimal::try_from_i128_with_scale(val, *scale)?; + Ok(JsonValue::String(decimal.to_string())) + } + _ => Err(Error::new( + ErrorKind::DataInvalid, + "The iceberg type for decimal literal must be decimal.", + ))?, + }, + }, + (Literal::Struct(s), Type::Struct(struct_type)) => { + let mut id_and_value = Vec::with_capacity(struct_type.fields().len()); + for (value, field) in s.into_iter().zip(struct_type.fields()) { + let json = match value { + Some(val) => val.try_into_json(&field.field_type)?, + None => JsonValue::Null, + }; + id_and_value.push((field.id.to_string(), json)); + } + Ok(JsonValue::Object(JsonMap::from_iter(id_and_value))) + } + (Literal::List(list), Type::List(list_type)) => Ok(JsonValue::Array( + list.into_iter() + .map(|opt| match opt { + Some(literal) => literal.try_into_json(&list_type.element_field.field_type), + None => Ok(JsonValue::Null), + }) + .collect::>>()?, + )), + (Literal::Map(map), Type::Map(map_type)) => { + let mut object = JsonMap::with_capacity(2); + let mut json_keys = Vec::with_capacity(map.len()); + let mut json_values = Vec::with_capacity(map.len()); + for (key, value) in map.into_iter() { + json_keys.push(key.try_into_json(&map_type.key_field.field_type)?); + json_values.push(match value { + Some(literal) => literal.try_into_json(&map_type.value_field.field_type)?, + None => JsonValue::Null, + }); + } + object.insert("keys".to_string(), JsonValue::Array(json_keys)); + object.insert("values".to_string(), JsonValue::Array(json_values)); + Ok(JsonValue::Object(object)) + } + (value, r#type) => Err(Error::new( + ErrorKind::DataInvalid, + format!( + "The iceberg value {:?} doesn't fit to the iceberg type {}.", + value, r#type + ), + )), + } + } + + /// Convert Value to the any type + pub fn into_any(self) -> Box { + match self { + Literal::Primitive(prim) => match prim { + PrimitiveLiteral::Boolean(any) => Box::new(any), + PrimitiveLiteral::Int(any) => Box::new(any), + PrimitiveLiteral::Long(any) => Box::new(any), + PrimitiveLiteral::Float(any) => Box::new(any), + PrimitiveLiteral::Double(any) => Box::new(any), + PrimitiveLiteral::Date(any) => Box::new(any), + PrimitiveLiteral::Time(any) => Box::new(any), + PrimitiveLiteral::Timestamp(any) => Box::new(any), + PrimitiveLiteral::TimestampTZ(any) => Box::new(any), + PrimitiveLiteral::Fixed(any) => Box::new(any), + PrimitiveLiteral::Binary(any) => Box::new(any), + PrimitiveLiteral::String(any) => Box::new(any), + PrimitiveLiteral::UUID(any) => Box::new(any), + PrimitiveLiteral::Decimal(any) => Box::new(any), + }, + _ => unimplemented!(), + } + } +} + +mod date { + use chrono::{NaiveDate, NaiveDateTime}; + + pub(crate) fn date_to_days(date: &NaiveDate) -> i32 { + date.signed_duration_since( + // This is always the same and shouldn't fail + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + ) + .num_days() as i32 + } + + pub(crate) fn days_to_date(days: i32) -> NaiveDate { + // This shouldn't fail until the year 262000 + NaiveDateTime::from_timestamp_opt(days as i64 * 86_400, 0) + .unwrap() + .date() + } +} + +mod time { + use chrono::NaiveTime; + + pub(crate) fn time_to_microseconds(time: &NaiveTime) -> i64 { + time.signed_duration_since( + // This is always the same and shouldn't fail + NaiveTime::from_num_seconds_from_midnight_opt(0, 0).unwrap(), + ) + .num_microseconds() + .unwrap() + } + + pub(crate) fn microseconds_to_time(micros: i64) -> NaiveTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem as u32 * 1_000).unwrap() + } +} + +mod timestamp { + use chrono::NaiveDateTime; + + pub(crate) fn datetime_to_microseconds(time: &NaiveDateTime) -> i64 { + time.timestamp_micros() + } + + pub(crate) fn microseconds_to_datetime(micros: i64) -> NaiveDateTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + // This shouldn't fail until the year 262000 + NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1_000).unwrap() + } +} + +mod timestamptz { + use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; + + pub(crate) fn datetimetz_to_microseconds(time: &DateTime) -> i64 { + time.timestamp_micros() + } + + pub(crate) fn microseconds_to_datetimetz(micros: i64) -> DateTime { + let (secs, rem) = (micros / 1_000_000, micros % 1_000_000); + + Utc.from_utc_datetime( + // This shouldn't fail until the year 262000 + &NaiveDateTime::from_timestamp_opt(secs, rem as u32 * 1_000).unwrap(), + ) + } +} + +mod _serde { + use std::collections::BTreeMap; + + use crate::{ + spec::{PrimitiveType, Type, MAP_KEY_FIELD_NAME, MAP_VALUE_FIELD_NAME}, + Error, ErrorKind, + }; + + use super::{Literal, PrimitiveLiteral}; + use serde::{ + de::Visitor, + ser::{SerializeMap, SerializeSeq, SerializeStruct}, + Deserialize, Serialize, + }; + use serde_bytes::ByteBuf; + use serde_derive::Deserialize as DeserializeDerive; + use serde_derive::Serialize as SerializeDerive; + + #[derive(SerializeDerive, DeserializeDerive, Debug)] + #[serde(transparent)] + /// Raw literal representation used for serde. The serialize way is used for Avro serializer. + pub struct RawLiteral(RawLiteralEnum); + + impl RawLiteral { + /// Covert literal to raw literal. + pub fn try_from(literal: Literal, ty: &Type) -> Result { + Ok(Self(RawLiteralEnum::try_from(literal, ty)?)) + } + + /// Convert raw literal to literal. + pub fn try_into(self, ty: &Type) -> Result, Error> { + self.0.try_into(ty) + } + } + + #[derive(SerializeDerive, Clone, Debug)] + #[serde(untagged)] + enum RawLiteralEnum { + Null, + Boolean(bool), + Int(i32), + Long(i64), + Float(f32), + Double(f64), + String(String), + Bytes(ByteBuf), + List(List), + StringMap(StringMap), + Record(Record), + } + + #[derive(Clone, Debug)] + struct Record { + required: Vec<(String, RawLiteralEnum)>, + optional: Vec<(String, Option)>, + } + + impl Serialize for Record { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let len = self.required.len() + self.optional.len(); + let mut record = serializer.serialize_struct("", len)?; + for (k, v) in &self.required { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + for (k, v) in &self.optional { + record.serialize_field(Box::leak(k.clone().into_boxed_str()), &v)?; + } + record.end() + } + } + + #[derive(Clone, Debug)] + struct List { + list: Vec>, + required: bool, + } + + impl Serialize for List { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut seq = serializer.serialize_seq(Some(self.list.len()))?; + for value in &self.list { + if self.required { + seq.serialize_element(value.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "List element is required, element cannot be null", + ) + })?)?; + } else { + seq.serialize_element(&value)?; + } + } + seq.end() + } + } + + #[derive(Clone, Debug)] + struct StringMap { + raw: Vec<(String, Option)>, + required: bool, + } + + impl Serialize for StringMap { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + let mut map = serializer.serialize_map(Some(self.raw.len()))?; + for (k, v) in &self.raw { + if self.required { + map.serialize_entry( + k, + v.as_ref().ok_or_else(|| { + serde::ser::Error::custom( + "Map element is required, element cannot be null", + ) + })?, + )?; + } else { + map.serialize_entry(k, v)?; + } + } + map.end() + } + } + + impl<'de> Deserialize<'de> for RawLiteralEnum { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + struct RawLiteralVisitor; + impl<'de> Visitor<'de> for RawLiteralVisitor { + type Value = RawLiteralEnum; + + fn expecting(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { + formatter.write_str("expect") + } + + fn visit_bool(self, v: bool) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Boolean(v)) + } + + fn visit_i32(self, v: i32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Int(v)) + } + + fn visit_i64(self, v: i64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v)) + } + + /// Used in json + fn visit_u64(self, v: u64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Long(v as i64)) + } + + fn visit_f32(self, v: f32) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Float(v)) + } + + fn visit_f64(self, v: f64) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Double(v)) + } + + fn visit_str(self, v: &str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_bytes(self, v: &[u8]) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Bytes(ByteBuf::from(v))) + } + + fn visit_borrowed_str(self, v: &'de str) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::String(v.to_string())) + } + + fn visit_unit(self) -> Result + where + E: serde::de::Error, + { + Ok(RawLiteralEnum::Null) + } + + fn visit_map(self, mut map: A) -> Result + where + A: serde::de::MapAccess<'de>, + { + let mut required = Vec::new(); + while let Some(key) = map.next_key::()? { + let value = map.next_value::()?; + required.push((key, value)); + } + Ok(RawLiteralEnum::Record(Record { + required, + optional: Vec::new(), + })) + } + + fn visit_seq(self, mut seq: A) -> Result + where + A: serde::de::SeqAccess<'de>, + { + let mut list = Vec::new(); + while let Some(value) = seq.next_element::()? { + list.push(Some(value)); + } + Ok(RawLiteralEnum::List(List { + list, + // `required` only used in serialize, just set default in deserialize. + required: false, + })) + } + } + deserializer.deserialize_any(RawLiteralVisitor) + } + } + + impl RawLiteralEnum { + pub fn try_from(literal: Literal, ty: &Type) -> Result { + let raw = match literal { + Literal::Primitive(prim) => match prim { + super::PrimitiveLiteral::Boolean(v) => RawLiteralEnum::Boolean(v), + super::PrimitiveLiteral::Int(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Long(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Float(v) => RawLiteralEnum::Float(v.0), + super::PrimitiveLiteral::Double(v) => RawLiteralEnum::Double(v.0), + super::PrimitiveLiteral::Date(v) => RawLiteralEnum::Int(v), + super::PrimitiveLiteral::Time(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::Timestamp(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::TimestampTZ(v) => RawLiteralEnum::Long(v), + super::PrimitiveLiteral::String(v) => RawLiteralEnum::String(v), + super::PrimitiveLiteral::UUID(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.as_u128().to_be_bytes())) + } + super::PrimitiveLiteral::Fixed(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Binary(v) => RawLiteralEnum::Bytes(ByteBuf::from(v)), + super::PrimitiveLiteral::Decimal(v) => { + RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) + } + }, + Literal::Struct(r#struct) => { + let mut required = Vec::new(); + let mut optional = Vec::new(); + if let Type::Struct(struct_ty) = ty { + for (value, field) in r#struct.into_iter().zip(struct_ty.fields()) { + if field.required { + if let Some(value) = value { + required.push(( + field.name.clone(), + RawLiteralEnum::try_from(value, &field.field_type)?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "Can't convert null to required field", + )); + } + } else if let Some(value) = value { + optional.push(( + field.name.clone(), + Some(RawLiteralEnum::try_from(value, &field.field_type)?), + )); + } else { + optional.push((field.name.clone(), None)); + } + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a struct", ty), + )); + } + RawLiteralEnum::Record(Record { required, optional }) + } + Literal::List(list) => { + if let Type::List(list_ty) = ty { + let list = list + .into_iter() + .map(|v| { + v.map(|v| { + RawLiteralEnum::try_from(v, &list_ty.element_field.field_type) + }) + .transpose() + }) + .collect::>()?; + RawLiteralEnum::List(List { + list, + required: list_ty.element_field.required, + }) + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a list", ty), + )); + } + } + Literal::Map(map) => { + if let Type::Map(map_ty) = ty { + if let Type::Primitive(PrimitiveType::String) = *map_ty.key_field.field_type + { + let mut raw = Vec::with_capacity(map.len()); + for (k, v) in map { + if let Literal::Primitive(PrimitiveLiteral::String(k)) = k { + raw.push(( + k, + v.map(|v| { + RawLiteralEnum::try_from( + v, + &map_ty.value_field.field_type, + ) + }) + .transpose()?, + )); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + "literal type is inconsistent with type", + )); + } + } + RawLiteralEnum::StringMap(StringMap { + raw, + required: map_ty.value_field.required, + }) + } else { + let list = map.into_iter().map(|(k,v)| { + let raw_k = + RawLiteralEnum::try_from(k, &map_ty.key_field.field_type)?; + let raw_v = v + .map(|v| { + RawLiteralEnum::try_from(v, &map_ty.value_field.field_type) + }) + .transpose()?; + if map_ty.value_field.required { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + (MAP_VALUE_FIELD_NAME.to_string(), raw_v.ok_or_else(||Error::new(ErrorKind::DataInvalid, "Map value is required, value cannot be null"))?), + ], + optional: vec![], + }))) + } else { + Ok(Some(RawLiteralEnum::Record(Record { + required: vec![ + (MAP_KEY_FIELD_NAME.to_string(), raw_k), + ], + optional: vec![ + (MAP_VALUE_FIELD_NAME.to_string(), raw_v) + ], + }))) + } + }).collect::>()?; + RawLiteralEnum::List(List { + list, + required: true, + }) + } + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Type {} should be a map", ty), + )); + } + } + }; + Ok(raw) + } + + pub fn try_into(self, ty: &Type) -> Result, Error> { + let invalid_err = |v: &str| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: type mismatch", + v, ty + ), + ) + }; + let invalid_err_with_reason = |v: &str, reason: &str| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Unable to convert raw literal ({}) fail convert to type {} for: {}", + v, ty, reason + ), + ) + }; + match self { + RawLiteralEnum::Null => Ok(None), + RawLiteralEnum::Boolean(v) => Ok(Some(Literal::bool(v))), + RawLiteralEnum::Int(v) => match ty { + Type::Primitive(PrimitiveType::Int) => Ok(Some(Literal::int(v))), + Type::Primitive(PrimitiveType::Date) => Ok(Some(Literal::date(v))), + _ => Err(invalid_err("int")), + }, + RawLiteralEnum::Long(v) => match ty { + Type::Primitive(PrimitiveType::Long) => Ok(Some(Literal::long(v))), + Type::Primitive(PrimitiveType::Time) => Ok(Some(Literal::time(v))), + Type::Primitive(PrimitiveType::Timestamp) => Ok(Some(Literal::timestamp(v))), + Type::Primitive(PrimitiveType::Timestamptz) => { + Ok(Some(Literal::timestamptz(v))) + } + _ => Err(invalid_err("long")), + }, + RawLiteralEnum::Float(v) => match ty { + Type::Primitive(PrimitiveType::Float) => Ok(Some(Literal::float(v))), + _ => Err(invalid_err("float")), + }, + RawLiteralEnum::Double(v) => match ty { + Type::Primitive(PrimitiveType::Double) => Ok(Some(Literal::double(v))), + _ => Err(invalid_err("double")), + }, + RawLiteralEnum::String(v) => match ty { + Type::Primitive(PrimitiveType::String) => Ok(Some(Literal::string(v))), + _ => Err(invalid_err("string")), + }, + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now. + RawLiteralEnum::Bytes(_) => Err(invalid_err_with_reason( + "bytes", + "todo: rust avro doesn't support deserialize any bytes representation now", + )), + RawLiteralEnum::List(v) => { + match ty { + Type::List(ty) => Ok(Some(Literal::List( + v.list + .into_iter() + .map(|v| { + if let Some(v) = v { + v.try_into(&ty.element_field.field_type) + } else { + Ok(None) + } + }) + .collect::>()?, + ))), + Type::Map(map_ty) => { + let key_ty = map_ty.key_field.field_type.as_ref(); + let value_ty = map_ty.value_field.field_type.as_ref(); + let mut map = BTreeMap::new(); + for k_v in v.list { + let k_v = k_v.ok_or_else(|| invalid_err_with_reason("list","In deserialize, None will be represented as Some(RawLiteral::Null), all element in list must be valid"))?; + if let RawLiteralEnum::Record(Record { + required, + optional: _, + }) = k_v + { + if required.len() != 2 { + return Err(invalid_err_with_reason("list","Record must contains two element(key and value) of array")); + } + let mut key = None; + let mut value = None; + required.into_iter().for_each(|(k, v)| { + if k == MAP_KEY_FIELD_NAME { + key = Some(v); + } else if k == MAP_VALUE_FIELD_NAME { + value = Some(v); + } + }); + match (key, value) { + (Some(k), Some(v)) => { + let key = k.try_into(key_ty)?.ok_or_else(|| { + invalid_err_with_reason( + "list", + "Key element in Map must be valid", + ) + })?; + let value = v.try_into(value_ty)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "list", + "Value element is required in this Map", + )); + } + map.insert(key, value); + } + _ => return Err(invalid_err_with_reason( + "list", + "The elements of record in list are not key and value", + )), + } + } else { + return Err(invalid_err_with_reason( + "list", + "Map should represented as record array.", + )); + } + } + Ok(Some(Literal::Map(map))) + } + _ => Err(invalid_err("list")), + } + } + RawLiteralEnum::Record(Record { + required, + optional: _, + }) => match ty { + Type::Struct(struct_ty) => { + let iters: Vec> = required + .into_iter() + .map(|(field_name, value)| { + let field = struct_ty + .field_by_name(field_name.as_str()) + .ok_or_else(|| { + invalid_err_with_reason( + "record", + &format!("field {} is not exist", &field_name), + ) + })?; + let value = value.try_into(&field.field_type)?; + Ok(value) + }) + .collect::>()?; + Ok(Some(Literal::Struct(super::Struct::from_iter(iters)))) + } + Type::Map(map_ty) => { + if *map_ty.key_field.field_type != Type::Primitive(PrimitiveType::String) { + return Err(invalid_err_with_reason( + "record", + "Map key must be string", + )); + } + let mut map = BTreeMap::new(); + for (k, v) in required { + let value = v.try_into(&map_ty.value_field.field_type)?; + if map_ty.value_field.required && value.is_none() { + return Err(invalid_err_with_reason( + "record", + "Value element is required in this Map", + )); + } + map.insert(Literal::string(k), value); + } + Ok(Some(Literal::Map(map))) + } + _ => Err(invalid_err("record")), + }, + RawLiteralEnum::StringMap(_) => Err(invalid_err("string map")), + } + } + } +} + +#[cfg(test)] +mod tests { + + use apache_avro::{to_value, types::Value}; + + use crate::{ + avro::schema_to_avro_schema, + spec::{ + datatypes::{ListType, MapType, NestedField, StructType}, + Schema, + }, + }; + + use super::*; + + fn check_json_serde(json: &str, expected_literal: Literal, expected_type: &Type) { + let raw_json_value = serde_json::from_str::(json).unwrap(); + let desered_literal = + Literal::try_from_json(raw_json_value.clone(), expected_type).unwrap(); + assert_eq!(desered_literal, Some(expected_literal.clone())); + + let expected_json_value: JsonValue = expected_literal.try_into_json(expected_type).unwrap(); + let sered_json = serde_json::to_string(&expected_json_value).unwrap(); + let parsed_json_value = serde_json::from_str::(&sered_json).unwrap(); + + assert_eq!(parsed_json_value, raw_json_value); + } + + fn check_avro_bytes_serde(input: Vec, expected_literal: Literal, expected_type: &Type) { + let raw_schema = r#""bytes""#; + let schema = apache_avro::Schema::parse_str(raw_schema).unwrap(); + + let bytes = ByteBuf::from(input); + let literal = Literal::try_from_bytes(&bytes, expected_type).unwrap(); + assert_eq!(literal, expected_literal); + + let mut writer = apache_avro::Writer::new(&schema, Vec::new()); + writer.append_ser(ByteBuf::from(literal)).unwrap(); + let encoded = writer.into_inner().unwrap(); + let reader = apache_avro::Reader::with_schema(&schema, &*encoded).unwrap(); + + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = Literal::try_from_bytes(&result, expected_type).unwrap(); + assert_eq!(desered_literal, expected_literal); + } + } + + fn check_convert_with_avro(expected_literal: Literal, expected_type: &Type) { + let fields = vec![NestedField::required(1, "col", expected_type.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = + Literal::Struct(Struct::from_iter(vec![Some(expected_literal.clone())])); + + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + writer.append_ser(raw_literal).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + let result = apache_avro::from_value::(&record.unwrap()).unwrap(); + let desered_literal = result.try_into(&struct_type).unwrap().unwrap(); + assert_eq!(desered_literal, struct_literal); + } + } + + fn check_serialize_avro(literal: Literal, ty: &Type, expect_value: Value) { + let expect_value = Value::Record(vec![("col".to_string(), expect_value)]); + + let fields = vec![NestedField::required(1, "col", ty.clone()).into()]; + let schema = Schema::builder() + .with_fields(fields.clone()) + .build() + .unwrap(); + let avro_schema = schema_to_avro_schema("test", &schema).unwrap(); + let struct_type = Type::Struct(StructType::new(fields)); + let struct_literal = Literal::Struct(Struct::from_iter(vec![Some(literal.clone())])); + let mut writer = apache_avro::Writer::new(&avro_schema, Vec::new()); + let raw_literal = RawLiteral::try_from(struct_literal.clone(), &struct_type).unwrap(); + let value = to_value(raw_literal) + .unwrap() + .resolve(&avro_schema) + .unwrap(); + writer.append_value_ref(&value).unwrap(); + let encoded = writer.into_inner().unwrap(); + + let reader = apache_avro::Reader::new(&*encoded).unwrap(); + for record in reader { + assert_eq!(record.unwrap(), expect_value); + } + } + + #[test] + fn json_boolean() { + let record = r#"true"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn json_int() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn json_long() { + let record = r#"32"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn json_float() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn json_double() { + let record = r#"1.0"#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn json_date() { + let record = r#""2017-11-16""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn json_time() { + let record = r#""22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn json_timestamp() { + let record = r#""2017-11-16T22:31:08.123456""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn json_timestamptz() { + let record = r#""2017-11-16T22:31:08.123456+00:00""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::TimestampTZ(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn json_string() { + let record = r#""iceberg""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn json_uuid() { + let record = r#""f79c3e09-677c-4bbd-a479-3f349cb785e7""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::UUID( + Uuid::parse_str("f79c3e09-677c-4bbd-a479-3f349cb785e7").unwrap(), + )), + &Type::Primitive(PrimitiveType::Uuid), + ); + } + + #[test] + fn json_decimal() { + let record = r#""14.20""#; + + check_json_serde( + record, + Literal::Primitive(PrimitiveLiteral::Decimal(1420)), + &Type::decimal(28, 2).unwrap(), + ); + } + + #[test] + fn json_struct() { + let record = r#"{"1": 1, "2": "bar", "3": null}"#; + + check_json_serde( + record, + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + #[test] + fn json_list() { + let record = r#"[1, 2, 3, null]"#; + + check_json_serde( + record, + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn json_map() { + let record = r#"{ "keys": ["a", "b", "c"], "values": [1, 2, null] }"#; + + check_json_serde( + record, + Literal::Map(BTreeMap::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_bytes_boolean() { + let bytes = vec![1u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + &Type::Primitive(PrimitiveType::Boolean), + ); + } + + #[test] + fn avro_bytes_int() { + let bytes = vec![32u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_bytes_long() { + let bytes = vec![32u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 0u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_bytes_float() { + let bytes = vec![0u8, 0u8, 128u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_bytes_double() { + let bytes = vec![0u8, 0u8, 0u8, 0u8, 0u8, 0u8, 240u8, 63u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_bytes_string() { + let bytes = vec![105u8, 99u8, 101u8, 98u8, 101u8, 114u8, 103u8]; + + check_avro_bytes_serde( + bytes, + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_int() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Int(32)), + &Type::Primitive(PrimitiveType::Int), + ); + } + + #[test] + fn avro_convert_test_long() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Long(32)), + &Type::Primitive(PrimitiveType::Long), + ); + } + + #[test] + fn avro_convert_test_float() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Float), + ); + } + + #[test] + fn avro_convert_test_double() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(1.0))), + &Type::Primitive(PrimitiveType::Double), + ); + } + + #[test] + fn avro_convert_test_string() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::String("iceberg".to_string())), + &Type::Primitive(PrimitiveType::String), + ); + } + + #[test] + fn avro_convert_test_date() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Date(17486)), + &Type::Primitive(PrimitiveType::Date), + ); + } + + #[test] + fn avro_convert_test_time() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Time(81068123456)), + &Type::Primitive(PrimitiveType::Time), + ); + } + + #[test] + fn avro_convert_test_timestamp() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::Timestamp(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamp), + ); + } + + #[test] + fn avro_convert_test_timestamptz() { + check_convert_with_avro( + Literal::Primitive(PrimitiveLiteral::TimestampTZ(1510871468123456)), + &Type::Primitive(PrimitiveType::Timestamptz), + ); + } + + #[test] + fn avro_convert_test_list() { + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + None, + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro( + Literal::List(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ]), + &Type::List(ListType { + element_field: NestedField::list_element( + 0, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_map() { + check_convert_with_avro( + Literal::Map(BTreeMap::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + (Literal::Primitive(PrimitiveLiteral::Int(3)), None), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + false, + ) + .into(), + }), + ); + + check_convert_with_avro( + Literal::Map(BTreeMap::from([ + ( + Literal::Primitive(PrimitiveLiteral::Int(1)), + Some(Literal::Primitive(PrimitiveLiteral::Long(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(2)), + Some(Literal::Primitive(PrimitiveLiteral::Long(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::Int(3)), + Some(Literal::Primitive(PrimitiveLiteral::Long(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Long), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_string_map() { + check_convert_with_avro( + Literal::Map(BTreeMap::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + None, + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + false, + ) + .into(), + }), + ); + + check_convert_with_avro( + Literal::Map(BTreeMap::from([ + ( + Literal::Primitive(PrimitiveLiteral::String("a".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("b".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(2))), + ), + ( + Literal::Primitive(PrimitiveLiteral::String("c".to_string())), + Some(Literal::Primitive(PrimitiveLiteral::Int(3))), + ), + ])), + &Type::Map(MapType { + key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + .into(), + value_field: NestedField::map_value_element( + 1, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + ); + } + + #[test] + fn avro_convert_test_record() { + check_convert_with_avro( + Literal::Struct(Struct::from_iter(vec![ + Some(Literal::Primitive(PrimitiveLiteral::Int(1))), + Some(Literal::Primitive(PrimitiveLiteral::String( + "bar".to_string(), + ))), + None, + ])), + &Type::Struct(StructType::new(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + ])), + ); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro don't support deserialize any bytes representation now: + // - binary + // - decimal + #[test] + fn avro_convert_test_binary_ser() { + let literal = Literal::Primitive(PrimitiveLiteral::Binary(vec![1, 2, 3, 4, 5])); + let ty = Type::Primitive(PrimitiveType::Binary); + let expect_value = Value::Bytes(vec![1, 2, 3, 4, 5]); + check_serialize_avro(literal, &ty, expect_value); + } + + #[test] + fn avro_convert_test_decimal_ser() { + let literal = Literal::decimal(12345); + let ty = Type::Primitive(PrimitiveType::Decimal { + precision: 9, + scale: 8, + }); + let expect_value = Value::Decimal(apache_avro::Decimal::from(12345_i128.to_be_bytes())); + check_serialize_avro(literal, &ty, expect_value); + } + + // # TODO:https://github.com/apache/iceberg-rust/issues/86 + // rust avro can't support to convert any byte-like type to fixed in avro now. + // - uuid ser/de + // - fixed ser/de +} diff --git a/crates/iceberg/src/table.rs b/crates/iceberg/src/table.rs new file mode 100644 index 0000000..e3260a8 --- /dev/null +++ b/crates/iceberg/src/table.rs @@ -0,0 +1,65 @@ +// 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. + +//! Table API for Apache Iceberg +use crate::io::FileIO; +use crate::scan::TableScanBuilder; +use crate::spec::{TableMetadata, TableMetadataRef}; +use crate::TableIdent; +use typed_builder::TypedBuilder; + +/// Table represents a table in the catalog. +#[derive(TypedBuilder, Debug)] +pub struct Table { + file_io: FileIO, + #[builder(default, setter(strip_option, into))] + metadata_location: Option, + #[builder(setter(into))] + metadata: TableMetadataRef, + identifier: TableIdent, +} + +impl Table { + /// Returns table identifier. + pub fn identifier(&self) -> &TableIdent { + &self.identifier + } + /// Returns current metadata. + pub fn metadata(&self) -> &TableMetadata { + &self.metadata + } + + /// Returns current metadata ref. + pub fn metadata_ref(&self) -> TableMetadataRef { + self.metadata.clone() + } + + /// Returns current metadata location. + pub fn metadata_location(&self) -> Option<&str> { + self.metadata_location.as_deref() + } + + /// Returns file io used in this table. + pub fn file_io(&self) -> &FileIO { + &self.file_io + } + + /// Creates a table scan. + pub fn scan(&self) -> TableScanBuilder<'_> { + TableScanBuilder::new(self) + } +} diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs new file mode 100644 index 0000000..165fb89 --- /dev/null +++ b/crates/iceberg/src/transaction.rs @@ -0,0 +1,370 @@ +// 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. + +//! This module contains transaction api. + +use crate::error::Result; +use crate::spec::{FormatVersion, NullOrder, SortDirection, SortField, SortOrder, Transform}; +use crate::table::Table; +use crate::TableUpdate::UpgradeFormatVersion; +use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; +use std::cmp::Ordering; +use std::collections::HashMap; +use std::mem::discriminant; + +/// Table transaction. +pub struct Transaction<'a> { + table: &'a Table, + updates: Vec, + requirements: Vec, +} + +impl<'a> Transaction<'a> { + /// Creates a new transaction. + pub fn new(table: &'a Table) -> Self { + Self { + table, + updates: vec![], + requirements: vec![], + } + } + + fn append_updates(&mut self, updates: Vec) -> Result<()> { + for update in &updates { + for up in &self.updates { + if discriminant(up) == discriminant(update) { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot apply update with same type at same time: {:?}", + update + ), + )); + } + } + } + self.updates.extend(updates); + Ok(()) + } + + fn append_requirements(&mut self, requirements: Vec) -> Result<()> { + self.requirements.extend(requirements); + Ok(()) + } + + /// Sets table to a new version. + pub fn upgrade_table_version(mut self, format_version: FormatVersion) -> Result { + let current_version = self.table.metadata().format_version(); + match current_version.cmp(&format_version) { + Ordering::Greater => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Cannot downgrade table version from {} to {}", + current_version, format_version + ), + )); + } + Ordering::Less => { + self.append_updates(vec![UpgradeFormatVersion { format_version }])?; + } + Ordering::Equal => { + // Do nothing. + } + } + Ok(self) + } + + /// Update table's property. + pub fn set_properties(mut self, props: HashMap) -> Result { + self.append_updates(vec![TableUpdate::SetProperties { updates: props }])?; + Ok(self) + } + + /// Creates replace sort order action. + pub fn replace_sort_order(self) -> ReplaceSortOrderAction<'a> { + ReplaceSortOrderAction { + tx: self, + sort_fields: vec![], + } + } + + /// Remove properties in table. + pub fn remove_properties(mut self, keys: Vec) -> Result { + self.append_updates(vec![TableUpdate::RemoveProperties { removals: keys }])?; + Ok(self) + } + + /// Commit transaction. + pub async fn commit(self, catalog: &impl Catalog) -> Result
{ + let table_commit = TableCommit::builder() + .ident(self.table.identifier().clone()) + .updates(self.updates) + .requirements(self.requirements) + .build(); + + catalog.update_table(table_commit).await + } +} + +/// Transaction action for replacing sort order. +pub struct ReplaceSortOrderAction<'a> { + tx: Transaction<'a>, + sort_fields: Vec, +} + +impl<'a> ReplaceSortOrderAction<'a> { + /// Adds a field for sorting in ascending order. + pub fn asc(self, name: &str, null_order: NullOrder) -> Result { + self.add_sort_field(name, SortDirection::Ascending, null_order) + } + + /// Adds a field for sorting in descending order. + pub fn desc(self, name: &str, null_order: NullOrder) -> Result { + self.add_sort_field(name, SortDirection::Descending, null_order) + } + + /// Finished building the action and apply it to the transaction. + pub fn apply(mut self) -> Result> { + let unbound_sort_order = SortOrder::builder() + .with_fields(self.sort_fields) + .build_unbound()?; + + let updates = vec![ + TableUpdate::AddSortOrder { + sort_order: unbound_sort_order, + }, + TableUpdate::SetDefaultSortOrder { sort_order_id: -1 }, + ]; + + let requirements = vec![ + TableRequirement::CurrentSchemaIdMatch { + current_schema_id: self.tx.table.metadata().current_schema().schema_id() as i64, + }, + TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: self + .tx + .table + .metadata() + .default_sort_order() + .ok_or(Error::new( + ErrorKind::Unexpected, + "default sort order impossible to be none", + ))? + .order_id, + }, + ]; + + self.tx.append_requirements(requirements)?; + self.tx.append_updates(updates)?; + Ok(self.tx) + } + + fn add_sort_field( + mut self, + name: &str, + sort_direction: SortDirection, + null_order: NullOrder, + ) -> Result { + let field_id = self + .tx + .table + .metadata() + .current_schema() + .field_id_by_name(name) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Cannot find field {} in table schema", name), + ) + })?; + + let sort_field = SortField::builder() + .source_id(field_id) + .transform(Transform::Identity) + .direction(sort_direction) + .null_order(null_order) + .build(); + + self.sort_fields.push(sort_field); + Ok(self) + } +} + +#[cfg(test)] +mod tests { + use crate::io::FileIO; + use crate::spec::{FormatVersion, TableMetadata}; + use crate::table::Table; + use crate::transaction::Transaction; + use crate::{TableIdent, TableRequirement, TableUpdate}; + use std::collections::HashMap; + use std::fs::File; + use std::io::BufReader; + + fn make_v1_table() -> Table { + let file = File::open(format!( + "{}/testdata/table_metadata/{}", + env!("CARGO_MANIFEST_DIR"), + "TableMetadataV1Valid.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, TableMetadata>(reader).unwrap(); + + Table::builder() + .metadata(resp) + .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .build() + } + + fn make_v2_table() -> Table { + let file = File::open(format!( + "{}/testdata/table_metadata/{}", + env!("CARGO_MANIFEST_DIR"), + "TableMetadataV2Valid.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, TableMetadata>(reader).unwrap(); + + Table::builder() + .metadata(resp) + .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .build() + } + + #[test] + fn test_upgrade_table_version_v1_to_v2() { + let table = make_v1_table(); + let tx = Transaction::new(&table); + let tx = tx.upgrade_table_version(FormatVersion::V2).unwrap(); + + assert_eq!( + vec![TableUpdate::UpgradeFormatVersion { + format_version: FormatVersion::V2 + }], + tx.updates + ); + } + + #[test] + fn test_upgrade_table_version_v2_to_v2() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx.upgrade_table_version(FormatVersion::V2).unwrap(); + + assert!( + tx.updates.is_empty(), + "Upgrade table to same version should not generate any updates" + ); + assert!( + tx.requirements.is_empty(), + "Upgrade table to same version should not generate any requirements" + ); + } + + #[test] + fn test_downgrade_table_version() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx.upgrade_table_version(FormatVersion::V1); + + assert!(tx.is_err(), "Downgrade table version should fail!"); + } + + #[test] + fn test_set_table_property() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx + .set_properties(HashMap::from([("a".to_string(), "b".to_string())])) + .unwrap(); + + assert_eq!( + vec![TableUpdate::SetProperties { + updates: HashMap::from([("a".to_string(), "b".to_string())]) + }], + tx.updates + ); + } + + #[test] + fn test_remove_property() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx + .remove_properties(vec!["a".to_string(), "b".to_string()]) + .unwrap(); + + assert_eq!( + vec![TableUpdate::RemoveProperties { + removals: vec!["a".to_string(), "b".to_string()] + }], + tx.updates + ); + } + + #[test] + fn test_replace_sort_order() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx.replace_sort_order().apply().unwrap(); + + assert_eq!( + vec![ + TableUpdate::AddSortOrder { + sort_order: Default::default() + }, + TableUpdate::SetDefaultSortOrder { sort_order_id: -1 } + ], + tx.updates + ); + + assert_eq!( + vec![ + TableRequirement::CurrentSchemaIdMatch { + current_schema_id: 1 + }, + TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: 3 + } + ], + tx.requirements + ); + } + + #[test] + fn test_do_same_update_in_same_transaction() { + let table = make_v2_table(); + let tx = Transaction::new(&table); + let tx = tx + .remove_properties(vec!["a".to_string(), "b".to_string()]) + .unwrap(); + + let tx = tx.remove_properties(vec!["c".to_string(), "d".to_string()]); + + assert!( + tx.is_err(), + "Should not allow to do same kinds update in same transaction" + ); + } +} diff --git a/crates/iceberg/src/transform/bucket.rs b/crates/iceberg/src/transform/bucket.rs new file mode 100644 index 0000000..beff0be --- /dev/null +++ b/crates/iceberg/src/transform/bucket.rs @@ -0,0 +1,245 @@ +// 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 arrow_array::ArrayRef; +use arrow_schema::{DataType, TimeUnit}; + +use super::TransformFunction; + +#[derive(Debug)] +pub struct Bucket { + mod_n: u32, +} + +impl Bucket { + pub fn new(mod_n: u32) -> Self { + Self { mod_n } + } +} + +impl Bucket { + /// When switch the hash function, we only need to change this function. + fn hash_bytes(mut v: &[u8]) -> i32 { + murmur3::murmur3_32(&mut v, 0).unwrap() as i32 + } + + fn hash_int(v: i32) -> i32 { + Self::hash_long(v as i64) + } + + fn hash_long(v: i64) -> i32 { + Self::hash_bytes(v.to_le_bytes().as_slice()) + } + + /// v is days from unix epoch + fn hash_date(v: i32) -> i32 { + Self::hash_int(v) + } + + /// v is microseconds from midnight + fn hash_time(v: i64) -> i32 { + Self::hash_long(v) + } + + /// v is microseconds from unix epoch + fn hash_timestamp(v: i64) -> i32 { + Self::hash_long(v) + } + + fn hash_str(s: &str) -> i32 { + Self::hash_bytes(s.as_bytes()) + } + + /// Decimal values are hashed using the minimum number of bytes required to hold the unscaled value as a two’s complement big-endian + /// ref: https://iceberg.apache.org/spec/#appendix-b-32-bit-hash-requirements + fn hash_decimal(v: i128) -> i32 { + let bytes = v.to_be_bytes(); + if let Some(start) = bytes.iter().position(|&x| x != 0) { + Self::hash_bytes(&bytes[start..]) + } else { + Self::hash_bytes(&[0]) + } + } + + /// def bucket_N(x) = (murmur3_x86_32_hash(x) & Integer.MAX_VALUE) % N + /// ref: https://iceberg.apache.org/spec/#partitioning + fn bucket_n(&self, v: i32) -> i32 { + (v & i32::MAX) % (self.mod_n as i32) + } +} + +impl TransformFunction for Bucket { + fn transform(&self, input: ArrayRef) -> crate::Result { + let res: arrow_array::Int32Array = match input.data_type() { + DataType::Int32 => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_int(v))), + DataType::Int64 => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_long(v))), + DataType::Decimal128(_, _) => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_decimal(v))), + DataType::Date32 => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_date(v))), + DataType::Time64(TimeUnit::Microsecond) => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_time(v))), + DataType::Timestamp(TimeUnit::Microsecond, _) => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| self.bucket_n(Self::hash_timestamp(v))), + DataType::Utf8 => arrow_array::Int32Array::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| self.bucket_n(Self::hash_str(v.unwrap()))), + ), + DataType::LargeUtf8 => arrow_array::Int32Array::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| self.bucket_n(Self::hash_str(v.unwrap()))), + ), + DataType::Binary => arrow_array::Int32Array::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| self.bucket_n(Self::hash_bytes(v.unwrap()))), + ), + DataType::LargeBinary => arrow_array::Int32Array::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| self.bucket_n(Self::hash_bytes(v.unwrap()))), + ), + DataType::FixedSizeBinary(_) => arrow_array::Int32Array::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| self.bucket_n(Self::hash_bytes(v.unwrap()))), + ), + _ => unreachable!("Unsupported data type: {:?}", input.data_type()), + }; + Ok(Arc::new(res)) + } +} + +#[cfg(test)] +mod test { + use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime}; + + use super::Bucket; + #[test] + fn test_hash() { + // test int + assert_eq!(Bucket::hash_int(34), 2017239379); + // test long + assert_eq!(Bucket::hash_long(34), 2017239379); + // test decimal + assert_eq!(Bucket::hash_decimal(1420), -500754589); + // test date + let date = NaiveDate::from_ymd_opt(2017, 11, 16).unwrap(); + assert_eq!( + Bucket::hash_date( + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + ), + -653330422 + ); + // test time + let time = NaiveTime::from_hms_opt(22, 31, 8).unwrap(); + assert_eq!( + Bucket::hash_time( + time.signed_duration_since(NaiveTime::from_hms_opt(0, 0, 0).unwrap()) + .num_microseconds() + .unwrap() + ), + -662762989 + ); + // test timestamp + let timestamp = + NaiveDateTime::parse_from_str("2017-11-16 22:31:08", "%Y-%m-%d %H:%M:%S").unwrap(); + assert_eq!( + Bucket::hash_timestamp( + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str("1970-01-01 00:00:00", "%Y-%m-%d %H:%M:%S") + .unwrap() + ) + .num_microseconds() + .unwrap() + ), + -2047944441 + ); + // test timestamp with tz + let timestamp = DateTime::parse_from_rfc3339("2017-11-16T14:31:08-08:00").unwrap(); + assert_eq!( + Bucket::hash_timestamp( + timestamp + .signed_duration_since( + DateTime::parse_from_rfc3339("1970-01-01T00:00:00-00:00").unwrap() + ) + .num_microseconds() + .unwrap() + ), + -2047944441 + ); + // test str + assert_eq!(Bucket::hash_str("iceberg"), 1210000089); + // test uuid + assert_eq!( + Bucket::hash_bytes( + [ + 0xF7, 0x9C, 0x3E, 0x09, 0x67, 0x7C, 0x4B, 0xBD, 0xA4, 0x79, 0x3F, 0x34, 0x9C, + 0xB7, 0x85, 0xE7 + ] + .as_ref() + ), + 1488055340 + ); + // test fixed and binary + assert_eq!( + Bucket::hash_bytes([0x00, 0x01, 0x02, 0x03].as_ref()), + -188683207 + ); + } +} diff --git a/crates/iceberg/src/transform/identity.rs b/crates/iceberg/src/transform/identity.rs new file mode 100644 index 0000000..d22c28f --- /dev/null +++ b/crates/iceberg/src/transform/identity.rs @@ -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 crate::Result; +use arrow_array::ArrayRef; + +use super::TransformFunction; + +/// Return identity array. +#[derive(Debug)] +pub struct Identity {} + +impl TransformFunction for Identity { + fn transform(&self, input: ArrayRef) -> Result { + Ok(input) + } +} diff --git a/crates/iceberg/src/transform/mod.rs b/crates/iceberg/src/transform/mod.rs new file mode 100644 index 0000000..dead9db --- /dev/null +++ b/crates/iceberg/src/transform/mod.rs @@ -0,0 +1,55 @@ +// 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. + +//! Transform function used to compute partition values. +use crate::{spec::Transform, Result}; +use arrow_array::ArrayRef; + +mod bucket; +mod identity; +mod temporal; +mod truncate; +mod void; + +/// TransformFunction is a trait that defines the interface for all transform functions. +pub trait TransformFunction: Send { + /// transform will take an input array and transform it into a new array. + /// The implementation of this function will need to check and downcast the input to specific + /// type. + fn transform(&self, input: ArrayRef) -> Result; +} + +/// BoxedTransformFunction is a boxed trait object of TransformFunction. +pub type BoxedTransformFunction = Box; + +/// create_transform_function creates a boxed trait object of TransformFunction from a Transform. +pub fn create_transform_function(transform: &Transform) -> Result { + match transform { + Transform::Identity => Ok(Box::new(identity::Identity {})), + Transform::Void => Ok(Box::new(void::Void {})), + Transform::Year => Ok(Box::new(temporal::Year {})), + Transform::Month => Ok(Box::new(temporal::Month {})), + Transform::Day => Ok(Box::new(temporal::Day {})), + Transform::Hour => Ok(Box::new(temporal::Hour {})), + Transform::Bucket(mod_n) => Ok(Box::new(bucket::Bucket::new(*mod_n))), + Transform::Truncate(width) => Ok(Box::new(truncate::Truncate::new(*width))), + Transform::Unknown => Err(crate::error::Error::new( + crate::ErrorKind::FeatureUnsupported, + "Transform Unknown is not implemented", + )), + } +} diff --git a/crates/iceberg/src/transform/temporal.rs b/crates/iceberg/src/transform/temporal.rs new file mode 100644 index 0000000..7b8deb1 --- /dev/null +++ b/crates/iceberg/src/transform/temporal.rs @@ -0,0 +1,412 @@ +// 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 super::TransformFunction; +use crate::{Error, ErrorKind, Result}; +use arrow_arith::{ + arity::binary, + temporal::{month_dyn, year_dyn}, +}; +use arrow_array::{ + types::Date32Type, Array, ArrayRef, Date32Array, Int32Array, TimestampMicrosecondArray, +}; +use arrow_schema::{DataType, TimeUnit}; +use chrono::Datelike; +use std::sync::Arc; + +/// The number of days since unix epoch. +const DAY_SINCE_UNIX_EPOCH: i32 = 719163; +/// Hour in one second. +const HOUR_PER_SECOND: f64 = 1.0_f64 / 3600.0_f64; +/// Day in one second. +const DAY_PER_SECOND: f64 = 1.0_f64 / 24.0_f64 / 3600.0_f64; +/// Year of unix epoch. +const UNIX_EPOCH_YEAR: i32 = 1970; + +/// Extract a date or timestamp year, as years from 1970 +#[derive(Debug)] +pub struct Year; + +impl TransformFunction for Year { + fn transform(&self, input: ArrayRef) -> Result { + let array = + year_dyn(&input).map_err(|err| Error::new(ErrorKind::Unexpected, format!("{err}")))?; + Ok(Arc::::new( + array + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| v - UNIX_EPOCH_YEAR), + )) + } +} + +/// Extract a date or timestamp month, as months from 1970-01-01 +#[derive(Debug)] +pub struct Month; + +impl TransformFunction for Month { + fn transform(&self, input: ArrayRef) -> Result { + let year_array = + year_dyn(&input).map_err(|err| Error::new(ErrorKind::Unexpected, format!("{err}")))?; + let year_array: Int32Array = year_array + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| 12 * (v - UNIX_EPOCH_YEAR)); + let month_array = + month_dyn(&input).map_err(|err| Error::new(ErrorKind::Unexpected, format!("{err}")))?; + Ok(Arc::::new( + binary( + month_array.as_any().downcast_ref::().unwrap(), + year_array.as_any().downcast_ref::().unwrap(), + // Compute month from 1970-01-01, so minus 1 here. + |a, b| a + b - 1, + ) + .unwrap(), + )) + } +} + +/// Extract a date or timestamp day, as days from 1970-01-01 +#[derive(Debug)] +pub struct Day; + +impl TransformFunction for Day { + fn transform(&self, input: ArrayRef) -> Result { + let res: Int32Array = match input.data_type() { + DataType::Timestamp(TimeUnit::Microsecond, _) => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| -> i32 { (v as f64 / 1000.0 / 1000.0 * DAY_PER_SECOND) as i32 }), + DataType::Date32 => { + input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| -> i32 { + Date32Type::to_naive_date(v).num_days_from_ce() - DAY_SINCE_UNIX_EPOCH + }) + } + _ => { + return Err(Error::new( + ErrorKind::Unexpected, + format!( + "Should not call internally for unsupported data type {:?}", + input.data_type() + ), + )) + } + }; + Ok(Arc::new(res)) + } +} + +/// Extract a timestamp hour, as hours from 1970-01-01 00:00:00 +#[derive(Debug)] +pub struct Hour; + +impl TransformFunction for Hour { + fn transform(&self, input: ArrayRef) -> Result { + let res: Int32Array = match input.data_type() { + DataType::Timestamp(TimeUnit::Microsecond, _) => input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| -> i32 { (v as f64 * HOUR_PER_SECOND / 1000.0 / 1000.0) as i32 }), + _ => { + return Err(Error::new( + ErrorKind::Unexpected, + format!( + "Should not call internally for unsupported data type {:?}", + input.data_type() + ), + )) + } + }; + Ok(Arc::new(res)) + } +} + +#[cfg(test)] +mod test { + use arrow_array::{ArrayRef, Date32Array, Int32Array, TimestampMicrosecondArray}; + use chrono::{NaiveDate, NaiveDateTime}; + use std::sync::Arc; + + use crate::transform::TransformFunction; + + #[test] + fn test_transform_years() { + let year = super::Year; + + // Test Date32 + let ori_date = vec![ + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2000, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2030, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2060, 1, 1).unwrap(), + ]; + let date_array: ArrayRef = Arc::new(Date32Array::from( + ori_date + .into_iter() + .map(|date| { + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + }) + .collect::>(), + )); + let res = year.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), 0); + assert_eq!(res.value(1), 30); + assert_eq!(res.value(2), 60); + assert_eq!(res.value(3), 90); + + // Test TimestampMicrosecond + let ori_timestamp = vec![ + NaiveDateTime::parse_from_str("1970-01-01 12:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2000-01-01 19:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2030-01-01 10:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2060-01-01 11:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + ]; + let date_array: ArrayRef = Arc::new(TimestampMicrosecondArray::from( + ori_timestamp + .into_iter() + .map(|timestamp| { + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str( + "1970-01-01 00:00:00.0", + "%Y-%m-%d %H:%M:%S.%f", + ) + .unwrap(), + ) + .num_microseconds() + .unwrap() + }) + .collect::>(), + )); + let res = year.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), 0); + assert_eq!(res.value(1), 30); + assert_eq!(res.value(2), 60); + assert_eq!(res.value(3), 90); + } + + #[test] + fn test_transform_months() { + let month = super::Month; + + // Test Date32 + let ori_date = vec![ + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2000, 4, 1).unwrap(), + NaiveDate::from_ymd_opt(2030, 7, 1).unwrap(), + NaiveDate::from_ymd_opt(2060, 10, 1).unwrap(), + ]; + let date_array: ArrayRef = Arc::new(Date32Array::from( + ori_date + .into_iter() + .map(|date| { + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + }) + .collect::>(), + )); + let res = month.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), 0); + assert_eq!(res.value(1), 30 * 12 + 3); + assert_eq!(res.value(2), 60 * 12 + 6); + assert_eq!(res.value(3), 90 * 12 + 9); + + // Test TimestampMicrosecond + let ori_timestamp = vec![ + NaiveDateTime::parse_from_str("1970-01-01 12:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2000-04-01 19:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2030-07-01 10:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2060-10-01 11:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + ]; + let date_array: ArrayRef = Arc::new(TimestampMicrosecondArray::from( + ori_timestamp + .into_iter() + .map(|timestamp| { + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str( + "1970-01-01 00:00:00.0", + "%Y-%m-%d %H:%M:%S.%f", + ) + .unwrap(), + ) + .num_microseconds() + .unwrap() + }) + .collect::>(), + )); + let res = month.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), 0); + assert_eq!(res.value(1), 30 * 12 + 3); + assert_eq!(res.value(2), 60 * 12 + 6); + assert_eq!(res.value(3), 90 * 12 + 9); + } + + #[test] + fn test_transform_days() { + let day = super::Day; + let ori_date = vec![ + NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(), + NaiveDate::from_ymd_opt(2000, 4, 1).unwrap(), + NaiveDate::from_ymd_opt(2030, 7, 1).unwrap(), + NaiveDate::from_ymd_opt(2060, 10, 1).unwrap(), + ]; + let expect_day = ori_date + .clone() + .into_iter() + .map(|data| { + data.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + }) + .collect::>(); + + // Test Date32 + let date_array: ArrayRef = Arc::new(Date32Array::from( + ori_date + .into_iter() + .map(|date| { + date.signed_duration_since(NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()) + .num_days() as i32 + }) + .collect::>(), + )); + let res = day.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), expect_day[0]); + assert_eq!(res.value(1), expect_day[1]); + assert_eq!(res.value(2), expect_day[2]); + assert_eq!(res.value(3), expect_day[3]); + + // Test TimestampMicrosecond + let ori_timestamp = vec![ + NaiveDateTime::parse_from_str("1970-01-01 12:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2000-04-01 19:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2030-07-01 10:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2060-10-01 11:30:42.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + ]; + let date_array: ArrayRef = Arc::new(TimestampMicrosecondArray::from( + ori_timestamp + .into_iter() + .map(|timestamp| { + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str( + "1970-01-01 00:00:00.0", + "%Y-%m-%d %H:%M:%S.%f", + ) + .unwrap(), + ) + .num_microseconds() + .unwrap() + }) + .collect::>(), + )); + let res = day.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), expect_day[0]); + assert_eq!(res.value(1), expect_day[1]); + assert_eq!(res.value(2), expect_day[2]); + assert_eq!(res.value(3), expect_day[3]); + } + + #[test] + fn test_transform_hours() { + let hour = super::Hour; + let ori_timestamp = vec![ + NaiveDateTime::parse_from_str("1970-01-01 19:01:23.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2000-03-01 12:01:23.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2030-10-02 10:01:23.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + NaiveDateTime::parse_from_str("2060-09-01 05:03:23.123", "%Y-%m-%d %H:%M:%S.%f") + .unwrap(), + ]; + let expect_hour = ori_timestamp + .clone() + .into_iter() + .map(|timestamp| { + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str( + "1970-01-01 00:00:0.0", + "%Y-%m-%d %H:%M:%S.%f", + ) + .unwrap(), + ) + .num_hours() as i32 + }) + .collect::>(); + + // Test TimestampMicrosecond + let date_array: ArrayRef = Arc::new(TimestampMicrosecondArray::from( + ori_timestamp + .into_iter() + .map(|timestamp| { + timestamp + .signed_duration_since( + NaiveDateTime::parse_from_str( + "1970-01-01 00:00:0.0", + "%Y-%m-%d %H:%M:%S.%f", + ) + .unwrap(), + ) + .num_microseconds() + .unwrap() + }) + .collect::>(), + )); + let res = hour.transform(date_array).unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); + assert_eq!(res.len(), 4); + assert_eq!(res.value(0), expect_hour[0]); + assert_eq!(res.value(1), expect_hour[1]); + assert_eq!(res.value(2), expect_hour[2]); + assert_eq!(res.value(3), expect_hour[3]); + } +} diff --git a/crates/iceberg/src/transform/truncate.rs b/crates/iceberg/src/transform/truncate.rs new file mode 100644 index 0000000..a8ebda8 --- /dev/null +++ b/crates/iceberg/src/transform/truncate.rs @@ -0,0 +1,218 @@ +// 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 arrow_array::ArrayRef; +use arrow_schema::DataType; + +use crate::Error; + +use super::TransformFunction; + +#[derive(Debug)] +pub struct Truncate { + width: u32, +} + +impl Truncate { + pub fn new(width: u32) -> Self { + Self { width } + } + + fn truncate_str_by_char(s: &str, max_chars: usize) -> &str { + match s.char_indices().nth(max_chars) { + None => s, + Some((idx, _)) => &s[..idx], + } + } +} + +impl TransformFunction for Truncate { + fn transform(&self, input: ArrayRef) -> crate::Result { + match input.data_type() { + DataType::Int32 => { + let width: i32 = self.width.try_into().map_err(|_| { + Error::new( + crate::ErrorKind::DataInvalid, + "width is failed to convert to i32 when truncate Int32Array", + ) + })?; + let res: arrow_array::Int32Array = input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| v - v.rem_euclid(width)); + Ok(Arc::new(res)) + } + DataType::Int64 => { + let width = self.width as i64; + let res: arrow_array::Int64Array = input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| v - (((v % width) + width) % width)); + Ok(Arc::new(res)) + } + DataType::Decimal128(precision, scale) => { + let width = self.width as i128; + let res: arrow_array::Decimal128Array = input + .as_any() + .downcast_ref::() + .unwrap() + .unary(|v| v - (((v % width) + width) % width)) + .with_precision_and_scale(*precision, *scale) + .map_err(|err| Error::new(crate::ErrorKind::Unexpected, format!("{err}")))?; + Ok(Arc::new(res)) + } + DataType::Utf8 => { + let len = self.width as usize; + let res: arrow_array::StringArray = arrow_array::StringArray::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.map(|v| Self::truncate_str_by_char(v, len))), + ); + Ok(Arc::new(res)) + } + DataType::LargeUtf8 => { + let len = self.width as usize; + let res: arrow_array::LargeStringArray = arrow_array::LargeStringArray::from_iter( + input + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|v| v.map(|v| Self::truncate_str_by_char(v, len))), + ); + Ok(Arc::new(res)) + } + _ => unreachable!("Truncate transform only supports (int,long,decimal,string) types"), + } + } +} + +#[cfg(test)] +mod test { + use std::sync::Arc; + + use arrow_array::{ + builder::PrimitiveBuilder, types::Decimal128Type, Decimal128Array, Int32Array, Int64Array, + }; + + use crate::transform::TransformFunction; + + // Test case ref from: https://iceberg.apache.org/spec/#truncate-transform-details + #[test] + fn test_truncate_simple() { + // test truncate int + let input = Arc::new(Int32Array::from(vec![1, -1])); + let res = super::Truncate::new(10).transform(input).unwrap(); + assert_eq!( + res.as_any().downcast_ref::().unwrap().value(0), + 0 + ); + assert_eq!( + res.as_any().downcast_ref::().unwrap().value(1), + -10 + ); + + // test truncate long + let input = Arc::new(Int64Array::from(vec![1, -1])); + let res = super::Truncate::new(10).transform(input).unwrap(); + assert_eq!( + res.as_any().downcast_ref::().unwrap().value(0), + 0 + ); + assert_eq!( + res.as_any().downcast_ref::().unwrap().value(1), + -10 + ); + + // test decimal + let mut buidler = PrimitiveBuilder::::new() + .with_precision_and_scale(20, 2) + .unwrap(); + buidler.append_value(1065); + let input = Arc::new(buidler.finish()); + let res = super::Truncate::new(50).transform(input).unwrap(); + assert_eq!( + res.as_any() + .downcast_ref::() + .unwrap() + .value(0), + 1050 + ); + + // test string + let input = Arc::new(arrow_array::StringArray::from(vec!["iceberg"])); + let res = super::Truncate::new(3).transform(input).unwrap(); + assert_eq!( + res.as_any() + .downcast_ref::() + .unwrap() + .value(0), + "ice" + ); + + // test large string + let input = Arc::new(arrow_array::LargeStringArray::from(vec!["iceberg"])); + let res = super::Truncate::new(3).transform(input).unwrap(); + assert_eq!( + res.as_any() + .downcast_ref::() + .unwrap() + .value(0), + "ice" + ); + } + + #[test] + fn test_string_truncate() { + let test1 = "イロハニホヘト"; + let test1_2_expected = "イロ"; + assert_eq!( + super::Truncate::truncate_str_by_char(test1, 2), + test1_2_expected + ); + + let test1_3_expected = "イロハ"; + assert_eq!( + super::Truncate::truncate_str_by_char(test1, 3), + test1_3_expected + ); + + let test2 = "щщаεはчωいにπάほхεろへσκζ"; + let test2_7_expected = "щщаεはчω"; + assert_eq!( + super::Truncate::truncate_str_by_char(test2, 7), + test2_7_expected + ); + + let test3 = "\u{FFFF}\u{FFFF}"; + assert_eq!(super::Truncate::truncate_str_by_char(test3, 2), test3); + + let test4 = "\u{10000}\u{10000}"; + let test4_1_expected = "\u{10000}"; + assert_eq!( + super::Truncate::truncate_str_by_char(test4, 1), + test4_1_expected + ); + } +} diff --git a/crates/iceberg/src/transform/void.rs b/crates/iceberg/src/transform/void.rs new file mode 100644 index 0000000..d419430 --- /dev/null +++ b/crates/iceberg/src/transform/void.rs @@ -0,0 +1,30 @@ +// 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 crate::Result; +use arrow_array::{new_null_array, ArrayRef}; + +use super::TransformFunction; + +#[derive(Debug)] +pub struct Void {} + +impl TransformFunction for Void { + fn transform(&self, input: ArrayRef) -> Result { + Ok(new_null_array(input.data_type(), input.len())) + } +} diff --git a/crates/iceberg/src/writer/file_writer/mod.rs b/crates/iceberg/src/writer/file_writer/mod.rs new file mode 100644 index 0000000..c8251fd --- /dev/null +++ b/crates/iceberg/src/writer/file_writer/mod.rs @@ -0,0 +1,39 @@ +// 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. + +//! This module contains the writer for data file format supported by iceberg: parquet, orc. + +use super::{CurrentFileStatus, DefaultOutput}; +use crate::Result; +use arrow_array::RecordBatch; +use futures::Future; + +/// File writer builder trait. +pub trait FileWriterBuilder: Send + Clone + 'static { + /// The associated file writer type. + type R: FileWriter; + /// Build file writer. + fn build(self) -> impl Future> + Send; +} + +/// File writer focus on writing record batch to different physical file format.(Such as parquet. orc) +pub trait FileWriter: Send + CurrentFileStatus + 'static { + /// Write record batch to file. + fn write(&mut self, batch: &RecordBatch) -> impl Future> + Send; + /// Close file writer. + fn close(self) -> impl Future> + Send; +} diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs new file mode 100644 index 0000000..ac79d7b --- /dev/null +++ b/crates/iceberg/src/writer/mod.rs @@ -0,0 +1,35 @@ +// 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. + +//! The iceberg writer module. + +use crate::spec::DataFileBuilder; + +pub mod file_writer; + +type DefaultOutput = Vec; + +/// The current file status of iceberg writer. It implement for the writer which write a single +/// file. +pub trait CurrentFileStatus { + /// Get the current file path. + fn current_file_path(&self) -> String; + /// Get the current file row number. + fn current_row_num(&self) -> usize; + /// Get the current file written size. + fn current_written_size(&self) -> usize; +} diff --git a/crates/iceberg/testdata/avro_schema_manifest_entry.json b/crates/iceberg/testdata/avro_schema_manifest_entry.json new file mode 100644 index 0000000..876c5fa --- /dev/null +++ b/crates/iceberg/testdata/avro_schema_manifest_entry.json @@ -0,0 +1,286 @@ +{ + "type": "record", + "name": "manifest_entry", + "fields": [ + { + "name": "status", + "type": "int", + "field-id": 0 + }, + { + "name": "snapshot_id", + "type": [ + "null", + "long" + ], + "field-id": 1 + }, + { + "name": "data_file", + "type": { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "file_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 100 + }, + { + "name": "file_format", + "type": "string", + "doc": "File format name: avro, orc, or parquet", + "field-id": 101 + }, + { + "name": "partition", + "type": { + "type": "record", + "name": "r102", + "fields": [ + { + "field-id": 1000, + "name": "VendorID", + "type": [ + "null", + "int" + ] + }, + { + "field-id": 1001, + "name": "tpep_pickup_datetime", + "type": [ + "null", + { + "type": "int", + "logicalType": "date" + } + ] + } + ] + }, + "field-id": 102 + }, + { + "name": "record_count", + "type": "long", + "doc": "Number of records in the file", + "field-id": 103 + }, + { + "name": "file_size_in_bytes", + "type": "long", + "doc": "Total file size in bytes", + "field-id": 104 + }, + { + "name": "block_size_in_bytes", + "type": "long", + "field-id": 105 + }, + { + "name": "column_sizes", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k117_v118", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 117 + }, + { + "name": "value", + "type": "long", + "field-id": 118 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to total size on disk", + "field-id": 108 + }, + { + "name": "value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k119_v120", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 119 + }, + { + "name": "value", + "type": "long", + "field-id": 120 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to total count, including null and NaN", + "field-id": 109 + }, + { + "name": "null_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k121_v122", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 121 + }, + { + "name": "value", + "type": "long", + "field-id": 122 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to null value count", + "field-id": 110 + }, + { + "name": "nan_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k138_v139", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 138 + }, + { + "name": "value", + "type": "long", + "field-id": 139 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to number of NaN values in the column", + "field-id": 137 + }, + { + "name": "lower_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k126_v127", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 126 + }, + { + "name": "value", + "type": "bytes", + "field-id": 127 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to lower bound", + "field-id": 125 + }, + { + "name": "upper_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k129_v130", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 129 + }, + { + "name": "value", + "type": "bytes", + "field-id": 130 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to upper bound", + "field-id": 128 + }, + { + "name": "key_metadata", + "type": [ + "null", + "bytes" + ], + "doc": "Encryption key metadata blob", + "field-id": 131 + }, + { + "name": "split_offsets", + "type": [ + "null", + { + "type": "array", + "items": "long", + "element-id": 133 + } + ], + "doc": "Splittable offsets", + "field-id": 132 + }, + { + "name": "sort_order_id", + "type": [ + "null", + "int" + ], + "doc": "Sort order ID", + "field-id": 140 + } + ] + }, + "field-id": 2 + } + ] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/avro_schema_manifest_file_v1.json b/crates/iceberg/testdata/avro_schema_manifest_file_v1.json new file mode 100644 index 0000000..b185094 --- /dev/null +++ b/crates/iceberg/testdata/avro_schema_manifest_file_v1.json @@ -0,0 +1,139 @@ +{ + "type": "record", + "name": "manifest_file", + "fields": [ + { + "name": "manifest_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 500 + }, + { + "name": "manifest_length", + "type": "long", + "doc": "Total file size in bytes", + "field-id": 501 + }, + { + "name": "partition_spec_id", + "type": "int", + "doc": "Spec ID used to write", + "field-id": 502 + }, + { + "name": "added_snapshot_id", + "type": [ + "null", + "long" + ], + "doc": "Snapshot ID that added the manifest", + "default": null, + "field-id": 503 + }, + { + "name": "added_data_files_count", + "type": [ + "null", + "int" + ], + "doc": "Added entry count", + "field-id": 504 + }, + { + "name": "existing_data_files_count", + "type": [ + "null", + "int" + ], + "doc": "Existing entry count", + "field-id": 505 + }, + { + "name": "deleted_data_files_count", + "type": [ + "null", + "int" + ], + "doc": "Deleted entry count", + "field-id": 506 + }, + { + "name": "partitions", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "r508", + "fields": [ + { + "name": "contains_null", + "type": "boolean", + "doc": "True if any file has a null partition value", + "field-id": 509 + }, + { + "name": "contains_nan", + "type": [ + "null", + "boolean" + ], + "doc": "True if any file has a nan partition value", + "field-id": 518 + }, + { + "name": "lower_bound", + "type": [ + "null", + "bytes" + ], + "doc": "Partition lower bound for all files", + "field-id": 510 + }, + { + "name": "upper_bound", + "type": [ + "null", + "bytes" + ], + "doc": "Partition upper bound for all files", + "field-id": 511 + } + ] + }, + "element-id": 508 + } + ], + "doc": "Summary for each partition", + "field-id": 507 + }, + { + "name": "added_rows_count", + "type": [ + "null", + "long" + ], + "doc": "Added rows count", + "field-id": 512 + }, + { + "name": "existing_rows_count", + "type": [ + "null", + "long" + ], + "doc": "Existing rows count", + "field-id": 513 + }, + { + "name": "deleted_rows_count", + "type": [ + "null", + "long" + ], + "doc": "Deleted rows count", + "field-id": 514 + } + ] +} diff --git a/crates/iceberg/testdata/avro_schema_manifest_file_v2.json b/crates/iceberg/testdata/avro_schema_manifest_file_v2.json new file mode 100644 index 0000000..34b97b9 --- /dev/null +++ b/crates/iceberg/testdata/avro_schema_manifest_file_v2.json @@ -0,0 +1,141 @@ +{ + "type": "record", + "name": "manifest_file", + "fields": [ + { + "name": "manifest_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 500 + }, + { + "name": "manifest_length", + "type": "long", + "doc": "Total file size in bytes", + "field-id": 501 + }, + { + "name": "partition_spec_id", + "type": "int", + "doc": "Spec ID used to write", + "field-id": 502 + }, + { + "name": "content", + "type": "int", + "doc": "Contents of the manifest: 0=data, 1=deletes", + "field-id": 517 + }, + { + "name": "sequence_number", + "type": [ + "null", + "long" + ], + "doc": "Sequence number when the manifest was added", + "field-id": 515 + }, + { + "name": "min_sequence_number", + "type": [ + "null", + "long" + ], + "doc": "Lowest sequence number in the manifest", + "field-id": 516 + }, + { + "name": "added_snapshot_id", + "type": "long", + "doc": "Snapshot ID that added the manifest", + "field-id": 503 + }, + { + "name": "added_files_count", + "type": "int", + "doc": "Added entry count", + "field-id": 504 + }, + { + "name": "existing_files_count", + "type": "int", + "doc": "Existing entry count", + "field-id": 505 + }, + { + "name": "deleted_files_count", + "type": "int", + "doc": "Deleted entry count", + "field-id": 506 + }, + { + "name": "added_rows_count", + "type": "long", + "doc": "Added rows count", + "field-id": 512 + }, + { + "name": "existing_rows_count", + "type": "long", + "doc": "Existing rows count", + "field-id": 513 + }, + { + "name": "deleted_rows_count", + "type": "long", + "doc": "Deleted rows count", + "field-id": 514 + }, + { + "name": "partitions", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "r508", + "fields": [ + { + "name": "contains_null", + "type": "boolean", + "doc": "True if any file has a null partition value", + "field-id": 509 + }, + { + "name": "contains_nan", + "type": [ + "null", + "boolean" + ], + "doc": "True if any file has a nan partition value", + "field-id": 518 + }, + { + "name": "lower_bound", + "type": [ + "null", + "bytes" + ], + "doc": "Partition lower bound for all files", + "field-id": 510 + }, + { + "name": "upper_bound", + "type": [ + "null", + "bytes" + ], + "doc": "Partition upper bound for all files", + "field-id": 511 + } + ] + }, + "element-id": 508 + } + ], + "doc": "Summary for each partition", + "field-id": 507 + } + ] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/example_table_metadata_v2.json b/crates/iceberg/testdata/example_table_metadata_v2.json new file mode 100644 index 0000000..809c355 --- /dev/null +++ b/crates/iceberg/testdata/example_table_metadata_v2.json @@ -0,0 +1,61 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "{{ table_location }}", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + {"type": "struct", "schema-id": 0, "fields": [{"id": 1, "name": "x", "required": true, "type": "long"}]}, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [1, 2], + "fields": [ + {"id": 1, "name": "x", "required": true, "type": "long"}, + {"id": 2, "name": "y", "required": true, "type": "long", "doc": "comment"}, + {"id": 3, "name": "z", "required": true, "type": "long"} + ] + } + ], + "default-spec-id": 0, + "partition-specs": [{"spec-id": 0, "fields": [{"name": "x", "transform": "identity", "source-id": 1, "field-id": 1000}]}], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + {"transform": "identity", "source-id": 2, "direction": "asc", "null-order": "nulls-first"}, + {"transform": "bucket[4]", "source-id": 3, "direction": "desc", "null-order": "nulls-last"} + ] + } + ], + "properties": {"read.split.target.size": "134217728"}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": {"operation": "append"}, + "manifest-list": "{{ manifest_list_1_location }}" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": {"operation": "append"}, + "manifest-list": "{{ manifest_list_2_location }}", + "schema-id": 1 + } + ], + "snapshot-log": [ + {"snapshot-id": 3051729675574597004, "timestamp-ms": 1515100955770}, + {"snapshot-id": 3055729675574597004, "timestamp-ms": 1555100955770} + ], + "metadata-log": [{"metadata-file": "{{ table_metadata_1_location }}", "timestamp-ms": 1515100}], + "refs": {"test": {"snapshot-id": 3051729675574597004, "type": "tag", "max-ref-age-ms": 10000000}} +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json b/crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json new file mode 100644 index 0000000..0633a71 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json @@ -0,0 +1,36 @@ +{ + "format-version": 3, + "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", + "location": "s3://bucket/test/location", + "last-updated-ms": 1602638573874, + "last-sequence-number": 0, + "last-column-id": 3, + "schema": { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + "partition-spec": [], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json b/crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json new file mode 100644 index 0000000..0b55d51 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json @@ -0,0 +1,42 @@ +{ + "format-version": 1, + "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", + "location": "s3://bucket/test/location", + "last-updated-ms": 1602638573874, + "last-column-id": 3, + "schema": { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + "partition-spec": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json new file mode 100644 index 0000000..d010785 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json @@ -0,0 +1,88 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 2, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json new file mode 100644 index 0000000..31c2b4c --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json @@ -0,0 +1,73 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [{ + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json new file mode 100644 index 0000000..3ab0a7a --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json @@ -0,0 +1,67 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [{ + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }], + "partition-spec": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ], + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json new file mode 100644 index 0000000..3754354 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json @@ -0,0 +1,71 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "schema": { + "type": "struct", + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }, + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json new file mode 100644 index 0000000..fbbcf41 --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json @@ -0,0 +1,54 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [{ + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + }], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "properties": {}, + "current-snapshot-id": -1, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json new file mode 100644 index 0000000..0dc89de --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json @@ -0,0 +1,122 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [ + 1, + 2 + ], + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/1.avro" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + } + ], + "snapshot-log": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770 + }, + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770 + } + ], + "metadata-log": [] +} \ No newline at end of file diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json b/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json new file mode 100644 index 0000000..529b10d --- /dev/null +++ b/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json @@ -0,0 +1,71 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ] +} \ No newline at end of file diff --git a/crates/rest/Cargo.toml b/crates/rest/Cargo.toml new file mode 100644 index 0000000..6db3060 --- /dev/null +++ b/crates/rest/Cargo.toml @@ -0,0 +1,48 @@ +# 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-rest" +version = { workspace = true } +edition = { workspace = true } +rust-version = { workspace = true } + +categories = ["database"] +description = "Apache Iceberg Rust REST API" +repository = { workspace = true } +license = { workspace = true } +keywords = ["iceberg", "rest", "catalog"] + +[dependencies] +# async-trait = { workspace = true } +async-trait = { workspace = true } +chrono = { workspace = true } +iceberg = { workspace = true } +log = "0.4.20" +reqwest = { workspace = true } +serde = { workspace = true } +serde_derive = { workspace = true } +serde_json = { workspace = true } +typed-builder = { workspace = true } +urlencoding = { workspace = true } +uuid = { workspace = true, features = ["v4"] } + +[dev-dependencies] +iceberg_test_utils = { path = "../test_utils", features = ["tests"] } +mockito = { workspace = true } +port_scanner = { workspace = true } +tokio = { workspace = true } diff --git a/crates/rest/DEPENDENCIES.rust.tsv b/crates/rest/DEPENDENCIES.rust.tsv new file mode 100644 index 0000000..6061ce8 --- /dev/null +++ b/crates/rest/DEPENDENCIES.rust.tsv @@ -0,0 +1,295 @@ +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT OpenSSL Unicode-DFS-2016 Unlicense Zlib +addr2line@0.21.0 X X +adler@1.0.2 X X X +adler32@1.2.0 X +ahash@0.8.6 X X +aho-corasick@1.1.2 X X +android-tzdata@0.1.1 X X +android_system_properties@0.1.5 X X +anstream@0.6.11 X X +anstyle@1.0.4 X X +anstyle-parse@0.2.3 X X +anstyle-query@1.0.2 X X +anstyle-wincon@3.0.2 X X +anyhow@1.0.77 X X +apache-avro@0.16.0 X +arrayvec@0.7.4 X X +arrow-arith@49.0.0 X +arrow-array@49.0.0 X +arrow-buffer@49.0.0 X +arrow-data@49.0.0 X +arrow-schema@49.0.0 X +async-compat@0.2.3 X X +async-trait@0.1.75 X X +autocfg@1.1.0 X X +backon@0.4.1 X +backtrace@0.3.69 X X +base64@0.21.5 X X +base64ct@1.6.0 X X +bimap@0.6.3 X X +bitflags@1.3.2 X X +bitflags@2.4.1 X X +bitvec@1.0.1 X +block-buffer@0.10.4 X X +bumpalo@3.14.0 X X +byteorder@1.5.0 X X +bytes@1.5.0 X +cc@1.0.83 X X +cfg-if@1.0.0 X X +chrono@0.4.31 X X +colorchoice@1.0.0 X X +const-oid@0.9.6 X X +const-random@0.1.17 X X +const-random-macro@0.1.16 X X +core-foundation@0.9.4 X X +core-foundation-sys@0.8.6 X X +core2@0.4.0 X X +cpufeatures@0.2.11 X X +crc32fast@1.3.2 X X +crunchy@0.2.2 X +crypto-common@0.1.6 X X +darling@0.14.4 X +darling@0.20.3 X +darling_core@0.14.4 X +darling_core@0.20.3 X +darling_macro@0.14.4 X +darling_macro@0.20.3 X +dary_heap@0.3.6 X X +der@0.7.8 X X +deranged@0.3.10 X X +derive_builder@0.13.0 X X +derive_builder_core@0.13.0 X X +derive_builder_macro@0.13.0 X X +digest@0.10.7 X X +dlv-list@0.5.2 X X +either@1.9.0 X X +encoding_rs@0.8.33 X X X +env_filter@0.1.0 X X +env_logger@0.11.0 X X +equivalent@1.0.1 X X +fastrand@1.9.0 X X +fastrand@2.0.1 X X +flagset@0.4.4 X +fnv@1.0.7 X X +foreign-types@0.3.2 X X +foreign-types-shared@0.1.1 X X +form_urlencoded@1.2.1 X X +funty@2.0.0 X +futures@0.3.30 X X +futures-channel@0.3.30 X X +futures-core@0.3.30 X X +futures-executor@0.3.30 X X +futures-io@0.3.30 X X +futures-macro@0.3.30 X X +futures-sink@0.3.30 X X +futures-task@0.3.30 X X +futures-util@0.3.30 X X +generic-array@0.14.7 X +getrandom@0.2.11 X X +gimli@0.28.1 X X +h2@0.3.22 X +half@2.3.1 X X +hashbrown@0.13.2 X X +hashbrown@0.14.3 X X +heck@0.4.1 X X +hermit-abi@0.3.3 X X +hex@0.4.3 X X +hmac@0.12.1 X X +home@0.5.9 X X +http@0.2.11 X X +http-body@0.4.6 X +httparse@1.8.0 X X +httpdate@1.0.3 X X +humantime@2.1.0 X X +hyper@0.14.28 X +hyper-rustls@0.24.2 X X X +hyper-tls@0.5.0 X X +iana-time-zone@0.1.58 X X +iana-time-zone-haiku@0.1.2 X X +iceberg@0.2.0 X +iceberg-catalog-rest@0.2.0 X +iceberg_test_utils@0.2.0 X +ident_case@1.0.1 X X +idna@0.5.0 X X +indexmap@2.1.0 X X +instant@0.1.12 X +ipnet@2.9.0 X X +itertools@0.12.0 X X +itoa@1.0.10 X X +js-sys@0.3.66 X X +jsonwebtoken@9.2.0 X +lazy_static@1.4.0 X X +libc@0.2.151 X X +libflate@2.0.0 X +libflate_lz77@2.0.0 X +libm@0.2.8 X X +linux-raw-sys@0.4.12 X X X +lock_api@0.4.11 X X +log@0.4.20 X X +md-5@0.10.6 X X +memchr@2.6.4 X X +mime@0.3.17 X X +miniz_oxide@0.7.1 X X X +mio@0.8.10 X +murmur3@0.5.2 X X +native-tls@0.2.11 X X +num@0.4.1 X X +num-bigint@0.4.4 X X +num-bigint-dig@0.8.4 X X +num-complex@0.4.4 X X +num-integer@0.1.45 X X +num-iter@0.1.43 X X +num-rational@0.4.1 X X +num-traits@0.2.17 X X +num_cpus@1.16.0 X X +object@0.32.2 X X +once_cell@1.19.0 X X +opendal@0.44.0 X +openssl@0.10.62 X +openssl-macros@0.1.1 X X +openssl-probe@0.1.5 X X +openssl-sys@0.9.98 X +ordered-float@4.2.0 X +ordered-multimap@0.7.1 X +parking_lot@0.12.1 X X +parking_lot_core@0.9.9 X X +pem@3.0.3 X +pem-rfc7468@0.7.0 X X +percent-encoding@2.3.1 X X +pin-project@1.1.3 X X +pin-project-internal@1.1.3 X X +pin-project-lite@0.2.13 X X +pin-utils@0.1.0 X X +pkcs1@0.7.5 X X +pkcs8@0.10.2 X X +pkg-config@0.3.28 X X +powerfmt@0.2.0 X X +ppv-lite86@0.2.17 X X +proc-macro2@1.0.71 X X +quad-rand@0.2.1 X +quick-xml@0.30.0 X +quick-xml@0.31.0 X +quote@1.0.33 X X +radium@0.7.0 X +rand@0.8.5 X X +rand_chacha@0.3.1 X X +rand_core@0.6.4 X X +redox_syscall@0.4.1 X +regex@1.10.2 X X +regex-automata@0.4.3 X X +regex-lite@0.1.5 X X +regex-syntax@0.8.2 X X +reqsign@0.14.6 X +reqwest@0.11.23 X X +ring@0.17.7 X +rle-decode-fast@1.0.3 X X +rsa@0.9.6 X X +rust-ini@0.20.0 X +rust_decimal@1.33.1 X +rustc-demangle@0.1.23 X X +rustix@0.38.28 X X X +rustls@0.21.10 X X X +rustls-native-certs@0.6.3 X X X +rustls-pemfile@1.0.4 X X X +rustls-webpki@0.101.7 X +rustversion@1.0.14 X X +ryu@1.0.16 X X +schannel@0.1.22 X +scopeguard@1.2.0 X X +sct@0.7.1 X X X +security-framework@2.9.2 X X +security-framework-sys@2.9.1 X X +serde@1.0.193 X X +serde_bytes@0.11.13 X X +serde_derive@1.0.193 X X +serde_json@1.0.108 X X +serde_repr@0.1.17 X X +serde_urlencoded@0.7.1 X X +serde_with@3.4.0 X X +serde_with_macros@3.4.0 X X +sha1@0.10.6 X X +sha2@0.10.8 X X +signal-hook-registry@1.4.1 X X +signature@2.2.0 X X +simple_asn1@0.6.2 X +slab@0.4.9 X +smallvec@1.11.2 X X +socket2@0.5.5 X X +spin@0.5.2 X +spin@0.9.8 X +spki@0.7.3 X X +strsim@0.10.0 X +strum@0.25.0 X +strum_macros@0.25.3 X +subtle@2.5.0 X +syn@1.0.109 X X +syn@2.0.43 X X +system-configuration@0.5.1 X X +system-configuration-sys@0.5.0 X X +tap@1.0.1 X +tempfile@3.8.1 X X +thiserror@1.0.52 X X +thiserror-impl@1.0.52 X X +time@0.3.31 X X +time-core@0.1.2 X X +time-macros@0.2.16 X X +tiny-keccak@2.0.2 X +tinyvec@1.6.0 X X X +tinyvec_macros@0.1.1 X X X +tokio@1.35.1 X +tokio-macros@2.2.0 X +tokio-native-tls@0.3.1 X +tokio-rustls@0.24.1 X X +tokio-util@0.7.10 X +tower-service@0.3.2 X +tracing@0.1.40 X +tracing-core@0.1.32 X +try-lock@0.2.5 X +typed-builder@0.16.2 X X +typed-builder@0.18.0 X X +typed-builder-macro@0.16.2 X X +typed-builder-macro@0.18.0 X X +typenum@1.17.0 X X +unicode-bidi@0.3.14 X X +unicode-ident@1.0.12 X X X +unicode-normalization@0.1.22 X X +untrusted@0.9.0 X +url@2.5.0 X X +urlencoding@2.1.3 X +utf8parse@0.2.1 X X +uuid@1.6.1 X X +vcpkg@0.2.15 X X +version_check@0.9.4 X X +want@0.3.1 X +wasi@0.11.0+wasi-snapshot-preview1 X X X +wasm-bindgen@0.2.89 X X +wasm-bindgen-backend@0.2.89 X X +wasm-bindgen-futures@0.4.39 X X +wasm-bindgen-macro@0.2.89 X X +wasm-bindgen-macro-support@0.2.89 X X +wasm-bindgen-shared@0.2.89 X X +wasm-streams@0.3.0 X X +web-sys@0.3.66 X X +windows-core@0.51.1 X X +windows-sys@0.48.0 X X +windows-sys@0.52.0 X X +windows-targets@0.48.5 X X +windows-targets@0.52.0 X X +windows_aarch64_gnullvm@0.48.5 X X +windows_aarch64_gnullvm@0.52.0 X X +windows_aarch64_msvc@0.48.5 X X +windows_aarch64_msvc@0.52.0 X X +windows_i686_gnu@0.48.5 X X +windows_i686_gnu@0.52.0 X X +windows_i686_msvc@0.48.5 X X +windows_i686_msvc@0.52.0 X X +windows_x86_64_gnu@0.48.5 X X +windows_x86_64_gnu@0.52.0 X X +windows_x86_64_gnullvm@0.48.5 X X +windows_x86_64_gnullvm@0.52.0 X X +windows_x86_64_msvc@0.48.5 X X +windows_x86_64_msvc@0.52.0 X X +winreg@0.50.0 X +wyz@0.5.1 X +zerocopy@0.7.32 X X X +zeroize@1.7.0 X X diff --git a/crates/rest/src/catalog.rs b/crates/rest/src/catalog.rs new file mode 100644 index 0000000..c10d904 --- /dev/null +++ b/crates/rest/src/catalog.rs @@ -0,0 +1,1604 @@ +// 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. + +//! This module contains rest catalog implementation. + +use std::collections::HashMap; + +use async_trait::async_trait; +use reqwest::header::{self, HeaderMap, HeaderName, HeaderValue}; +use reqwest::{Client, Request, Response, StatusCode}; +use serde::de::DeserializeOwned; +use typed_builder::TypedBuilder; +use urlencoding::encode; + +use crate::catalog::_serde::{ + CommitTableRequest, CommitTableResponse, CreateTableRequest, LoadTableResponse, +}; +use iceberg::io::FileIO; +use iceberg::table::Table; +use iceberg::Result; +use iceberg::{ + Catalog, Error, ErrorKind, Namespace, NamespaceIdent, TableCommit, TableCreation, TableIdent, +}; + +use self::_serde::{ + CatalogConfig, ErrorResponse, ListNamespaceResponse, ListTableResponse, NamespaceSerde, + RenameTableRequest, NO_CONTENT, OK, +}; + +const ICEBERG_REST_SPEC_VERSION: &str = "0.14.1"; +const CARGO_PKG_VERSION: &str = env!("CARGO_PKG_VERSION"); +const PATH_V1: &str = "v1"; + +/// Rest catalog configuration. +#[derive(Debug, TypedBuilder)] +pub struct RestCatalogConfig { + uri: String, + #[builder(default, setter(strip_option))] + warehouse: Option, + + #[builder(default)] + props: HashMap, +} + +impl RestCatalogConfig { + fn config_endpoint(&self) -> String { + [&self.uri, PATH_V1, "config"].join("/") + } + + fn namespaces_endpoint(&self) -> String { + [&self.uri, PATH_V1, "namespaces"].join("/") + } + + fn namespace_endpoint(&self, ns: &NamespaceIdent) -> String { + [&self.uri, PATH_V1, "namespaces", &ns.encode_in_url()].join("/") + } + + fn tables_endpoint(&self, ns: &NamespaceIdent) -> String { + [ + &self.uri, + PATH_V1, + "namespaces", + &ns.encode_in_url(), + "tables", + ] + .join("/") + } + + fn rename_table_endpoint(&self) -> String { + [&self.uri, PATH_V1, "tables", "rename"].join("/") + } + + fn table_endpoint(&self, table: &TableIdent) -> String { + [ + &self.uri, + PATH_V1, + "namespaces", + &table.namespace.encode_in_url(), + "tables", + encode(&table.name).as_ref(), + ] + .join("/") + } + + fn try_create_rest_client(&self) -> Result { + //TODO: We will add oauth, ssl config, sigv4 later + let headers = HeaderMap::from_iter([ + ( + header::CONTENT_TYPE, + HeaderValue::from_static("application/json"), + ), + ( + HeaderName::from_static("x-client-version"), + HeaderValue::from_static(ICEBERG_REST_SPEC_VERSION), + ), + ( + header::USER_AGENT, + HeaderValue::from_str(&format!("iceberg-rs/{}", CARGO_PKG_VERSION)).unwrap(), + ), + ]); + + Ok(HttpClient( + Client::builder().default_headers(headers).build()?, + )) + } +} + +#[derive(Debug)] +struct HttpClient(Client); + +impl HttpClient { + async fn query< + R: DeserializeOwned, + E: DeserializeOwned + Into, + const SUCCESS_CODE: u16, + >( + &self, + request: Request, + ) -> Result { + let resp = self.0.execute(request).await?; + + if resp.status().as_u16() == SUCCESS_CODE { + let text = resp.bytes().await?; + Ok(serde_json::from_slice::(&text).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to parse response from rest catalog server!", + ) + .with_context("json", String::from_utf8_lossy(&text)) + .with_source(e) + })?) + } else { + let text = resp.bytes().await?; + let e = serde_json::from_slice::(&text).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to parse response from rest catalog server!", + ) + .with_context("json", String::from_utf8_lossy(&text)) + .with_source(e) + })?; + Err(e.into()) + } + } + + async fn execute, const SUCCESS_CODE: u16>( + &self, + request: Request, + ) -> Result<()> { + let resp = self.0.execute(request).await?; + + if resp.status().as_u16() == SUCCESS_CODE { + Ok(()) + } else { + let code = resp.status(); + let text = resp.bytes().await?; + let e = serde_json::from_slice::(&text).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to parse response from rest catalog server!", + ) + .with_context("json", String::from_utf8_lossy(&text)) + .with_context("code", code.to_string()) + .with_source(e) + })?; + Err(e.into()) + } + } + + /// More generic logic handling for special cases like head. + async fn do_execute>( + &self, + request: Request, + handler: impl FnOnce(&Response) -> Option, + ) -> Result { + let resp = self.0.execute(request).await?; + + if let Some(ret) = handler(&resp) { + Ok(ret) + } else { + let code = resp.status(); + let text = resp.bytes().await?; + let e = serde_json::from_slice::(&text).map_err(|e| { + Error::new( + ErrorKind::Unexpected, + "Failed to parse response from rest catalog server!", + ) + .with_context("code", code.to_string()) + .with_context("json", String::from_utf8_lossy(&text)) + .with_source(e) + })?; + Err(e.into()) + } + } +} + +/// Rest catalog implementation. +#[derive(Debug)] +pub struct RestCatalog { + config: RestCatalogConfig, + client: HttpClient, +} + +#[async_trait] +impl Catalog for RestCatalog { + /// List namespaces from table. + async fn list_namespaces( + &self, + parent: Option<&NamespaceIdent>, + ) -> Result> { + let mut request = self.client.0.get(self.config.namespaces_endpoint()); + if let Some(ns) = parent { + request = request.query(&[("parent", ns.encode_in_url())]); + } + + let resp = self + .client + .query::(request.build()?) + .await?; + + resp.namespaces + .into_iter() + .map(NamespaceIdent::from_vec) + .collect::>>() + } + + /// Create a new namespace inside the catalog. + async fn create_namespace( + &self, + namespace: &NamespaceIdent, + properties: HashMap, + ) -> Result { + let request = self + .client + .0 + .post(self.config.namespaces_endpoint()) + .json(&NamespaceSerde { + namespace: namespace.as_ref().clone(), + properties: Some(properties), + }) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + + Namespace::try_from(resp) + } + + /// Get a namespace information from the catalog. + async fn get_namespace(&self, namespace: &NamespaceIdent) -> Result { + let request = self + .client + .0 + .get(self.config.namespace_endpoint(namespace)) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + Namespace::try_from(resp) + } + + /// Update a namespace inside the catalog. + /// + /// # Behavior + /// + /// The properties must be the full set of namespace. + async fn update_namespace( + &self, + _namespace: &NamespaceIdent, + _properties: HashMap, + ) -> Result<()> { + Err(Error::new( + ErrorKind::FeatureUnsupported, + "Updating namespace not supported yet!", + )) + } + + async fn namespace_exists(&self, ns: &NamespaceIdent) -> Result { + let request = self + .client + .0 + .head(self.config.namespace_endpoint(ns)) + .build()?; + + self.client + .do_execute::(request, |resp| match resp.status() { + StatusCode::NO_CONTENT => Some(true), + StatusCode::NOT_FOUND => Some(false), + _ => None, + }) + .await + } + + /// Drop a namespace from the catalog. + async fn drop_namespace(&self, namespace: &NamespaceIdent) -> Result<()> { + let request = self + .client + .0 + .delete(self.config.namespace_endpoint(namespace)) + .build()?; + + self.client + .execute::(request) + .await + } + + /// List tables from namespace. + async fn list_tables(&self, namespace: &NamespaceIdent) -> Result> { + let request = self + .client + .0 + .get(self.config.tables_endpoint(namespace)) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + + Ok(resp.identifiers) + } + + /// Create a new table inside the namespace. + async fn create_table( + &self, + namespace: &NamespaceIdent, + creation: TableCreation, + ) -> Result
{ + let table_ident = TableIdent::new(namespace.clone(), creation.name.clone()); + + let request = self + .client + .0 + .post(self.config.tables_endpoint(namespace)) + .json(&CreateTableRequest { + name: creation.name, + location: creation.location, + schema: creation.schema, + partition_spec: creation.partition_spec, + write_order: creation.sort_order, + // We don't support stage create yet. + stage_create: Some(false), + properties: if creation.properties.is_empty() { + None + } else { + Some(creation.properties) + }, + }) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + + let file_io = self.load_file_io(resp.metadata_location.as_deref(), resp.config)?; + + let table = Table::builder() + .identifier(table_ident) + .file_io(file_io) + .metadata(resp.metadata) + .metadata_location(resp.metadata_location.ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "Metadata location missing in create table response!", + ) + })?) + .build(); + + Ok(table) + } + + /// Load table from the catalog. + async fn load_table(&self, table: &TableIdent) -> Result
{ + let request = self + .client + .0 + .get(self.config.table_endpoint(table)) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + + let file_io = self.load_file_io(resp.metadata_location.as_deref(), resp.config)?; + + let table_builder = Table::builder() + .identifier(table.clone()) + .file_io(file_io) + .metadata(resp.metadata); + + if let Some(metadata_location) = resp.metadata_location { + Ok(table_builder.metadata_location(metadata_location).build()) + } else { + Ok(table_builder.build()) + } + } + + /// Drop a table from the catalog. + async fn drop_table(&self, table: &TableIdent) -> Result<()> { + let request = self + .client + .0 + .delete(self.config.table_endpoint(table)) + .build()?; + + self.client + .execute::(request) + .await + } + + /// Check if a table exists in the catalog. + async fn stat_table(&self, table: &TableIdent) -> Result { + let request = self + .client + .0 + .head(self.config.table_endpoint(table)) + .build()?; + + self.client + .do_execute::(request, |resp| match resp.status() { + StatusCode::NO_CONTENT => Some(true), + StatusCode::NOT_FOUND => Some(false), + _ => None, + }) + .await + } + + /// Rename a table in the catalog. + async fn rename_table(&self, src: &TableIdent, dest: &TableIdent) -> Result<()> { + let request = self + .client + .0 + .post(self.config.rename_table_endpoint()) + .json(&RenameTableRequest { + source: src.clone(), + destination: dest.clone(), + }) + .build()?; + + self.client + .execute::(request) + .await + } + + /// Update table. + async fn update_table(&self, mut commit: TableCommit) -> Result
{ + let request = self + .client + .0 + .post(self.config.table_endpoint(commit.identifier())) + .json(&CommitTableRequest { + identifier: commit.identifier().clone(), + requirements: commit.take_requirements(), + updates: commit.take_updates(), + }) + .build()?; + + let resp = self + .client + .query::(request) + .await?; + + let file_io = self.load_file_io(Some(&resp.metadata_location), None)?; + Ok(Table::builder() + .identifier(commit.identifier().clone()) + .file_io(file_io) + .metadata(resp.metadata) + .metadata_location(resp.metadata_location) + .build()) + } +} + +impl RestCatalog { + /// Creates a rest catalog from config. + pub async fn new(config: RestCatalogConfig) -> Result { + let mut catalog = Self { + client: config.try_create_rest_client()?, + config, + }; + + catalog.update_config().await?; + catalog.client = catalog.config.try_create_rest_client()?; + + Ok(catalog) + } + + async fn update_config(&mut self) -> Result<()> { + let mut request = self.client.0.get(self.config.config_endpoint()); + + if let Some(warehouse_location) = &self.config.warehouse { + request = request.query(&[("warehouse", warehouse_location)]); + } + + let config = self + .client + .query::(request.build()?) + .await?; + + let mut props = config.defaults; + props.extend(self.config.props.clone()); + props.extend(config.overrides); + + self.config.props = props; + + Ok(()) + } + + fn load_file_io( + &self, + metadata_location: Option<&str>, + extra_config: Option>, + ) -> Result { + let mut props = self.config.props.clone(); + if let Some(config) = extra_config { + props.extend(config); + } + + let file_io = match self.config.warehouse.as_deref().or(metadata_location) { + Some(url) => FileIO::from_path(url)?.with_props(props).build()?, + None => { + return Err(Error::new( + ErrorKind::Unexpected, + "Unable to load file io, neither warehouse nor metadata location is set!", + ))? + } + }; + + Ok(file_io) + } +} + +/// Requests and responses for rest api. +mod _serde { + use std::collections::HashMap; + + use serde_derive::{Deserialize, Serialize}; + + use iceberg::spec::{Schema, SortOrder, TableMetadata, UnboundPartitionSpec}; + use iceberg::{Error, ErrorKind, Namespace, TableIdent, TableRequirement, TableUpdate}; + + pub(super) const OK: u16 = 200u16; + pub(super) const NO_CONTENT: u16 = 204u16; + + #[derive(Clone, Debug, Serialize, Deserialize)] + pub(super) struct CatalogConfig { + pub(super) overrides: HashMap, + pub(super) defaults: HashMap, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct ErrorResponse { + error: ErrorModel, + } + + impl From for Error { + fn from(resp: ErrorResponse) -> Error { + resp.error.into() + } + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct ErrorModel { + pub(super) message: String, + pub(super) r#type: String, + pub(super) code: u16, + pub(super) stack: Option>, + } + + impl From for Error { + fn from(value: ErrorModel) -> Self { + let mut error = Error::new(ErrorKind::DataInvalid, value.message) + .with_context("type", value.r#type) + .with_context("code", format!("{}", value.code)); + + if let Some(stack) = value.stack { + error = error.with_context("stack", stack.join("\n")); + } + + error + } + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct OAuthError { + pub(super) error: String, + pub(super) error_description: Option, + pub(super) error_uri: Option, + } + + impl From for Error { + fn from(value: OAuthError) -> Self { + let mut error = Error::new( + ErrorKind::DataInvalid, + format!("OAuthError: {}", value.error), + ); + + if let Some(desc) = value.error_description { + error = error.with_context("description", desc); + } + + if let Some(uri) = value.error_uri { + error = error.with_context("uri", uri); + } + + error + } + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct NamespaceSerde { + pub(super) namespace: Vec, + pub(super) properties: Option>, + } + + impl TryFrom for super::Namespace { + type Error = Error; + fn try_from(value: NamespaceSerde) -> std::result::Result { + Ok(super::Namespace::with_properties( + super::NamespaceIdent::from_vec(value.namespace)?, + value.properties.unwrap_or_default(), + )) + } + } + + impl From<&Namespace> for NamespaceSerde { + fn from(value: &Namespace) -> Self { + Self { + namespace: value.name().as_ref().clone(), + properties: Some(value.properties().clone()), + } + } + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct ListNamespaceResponse { + pub(super) namespaces: Vec>, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct UpdateNamespacePropsRequest { + removals: Option>, + updates: Option>, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct UpdateNamespacePropsResponse { + updated: Vec, + removed: Vec, + missing: Option>, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct ListTableResponse { + pub(super) identifiers: Vec, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct RenameTableRequest { + pub(super) source: TableIdent, + pub(super) destination: TableIdent, + } + + #[derive(Debug, Deserialize)] + #[serde(rename_all = "kebab-case")] + pub(super) struct LoadTableResponse { + pub(super) metadata_location: Option, + pub(super) metadata: TableMetadata, + pub(super) config: Option>, + } + + #[derive(Debug, Serialize, Deserialize)] + #[serde(rename_all = "kebab-case")] + pub(super) struct CreateTableRequest { + pub(super) name: String, + pub(super) location: Option, + pub(super) schema: Schema, + pub(super) partition_spec: Option, + pub(super) write_order: Option, + pub(super) stage_create: Option, + pub(super) properties: Option>, + } + + #[derive(Debug, Serialize, Deserialize)] + pub(super) struct CommitTableRequest { + pub(super) identifier: TableIdent, + pub(super) requirements: Vec, + pub(super) updates: Vec, + } + + #[derive(Debug, Serialize, Deserialize)] + #[serde(rename_all = "kebab-case")] + pub(super) struct CommitTableResponse { + pub(super) metadata_location: String, + pub(super) metadata: TableMetadata, + } +} + +#[cfg(test)] +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, + }; + use iceberg::transaction::Transaction; + use mockito::{Mock, Server, ServerGuard}; + use std::fs::File; + use std::io::BufReader; + use std::sync::Arc; + use uuid::uuid; + + use super::*; + + #[tokio::test] + async fn test_update_config() { + let mut server = Server::new_async().await; + + let config_mock = server + .mock("GET", "/v1/config") + .with_status(200) + .with_body( + r#"{ + "overrides": { + "warehouse": "s3://iceberg-catalog" + }, + "defaults": {} + }"#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + assert_eq!( + catalog.config.props.get("warehouse"), + Some(&"s3://iceberg-catalog".to_string()) + ); + + config_mock.assert_async().await; + } + + async fn create_config_mock(server: &mut ServerGuard) -> Mock { + server + .mock("GET", "/v1/config") + .with_status(200) + .with_body( + r#"{ + "overrides": { + "warehouse": "s3://iceberg-catalog" + }, + "defaults": {} + }"#, + ) + .create_async() + .await + } + + #[tokio::test] + async fn test_list_namespace() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let list_ns_mock = server + .mock("GET", "/v1/namespaces") + .with_body( + r#"{ + "namespaces": [ + ["ns1", "ns11"], + ["ns2"] + ] + }"#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let namespaces = catalog.list_namespaces(None).await.unwrap(); + + let expected_ns = vec![ + NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), + NamespaceIdent::from_vec(vec!["ns2".to_string()]).unwrap(), + ]; + + assert_eq!(expected_ns, namespaces); + + config_mock.assert_async().await; + list_ns_mock.assert_async().await; + } + + #[tokio::test] + async fn test_create_namespace() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let create_ns_mock = server + .mock("POST", "/v1/namespaces") + .with_body( + r#"{ + "namespace": [ "ns1", "ns11"], + "properties" : { + "key1": "value1" + } + }"#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let namespaces = catalog + .create_namespace( + &NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), + HashMap::from([("key1".to_string(), "value1".to_string())]), + ) + .await + .unwrap(); + + let expected_ns = Namespace::with_properties( + NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), + HashMap::from([("key1".to_string(), "value1".to_string())]), + ); + + assert_eq!(expected_ns, namespaces); + + config_mock.assert_async().await; + create_ns_mock.assert_async().await; + } + + #[tokio::test] + async fn test_get_namespace() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let get_ns_mock = server + .mock("GET", "/v1/namespaces/ns1") + .with_body( + r#"{ + "namespace": [ "ns1"], + "properties" : { + "key1": "value1" + } + }"#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let namespaces = catalog + .get_namespace(&NamespaceIdent::new("ns1".to_string())) + .await + .unwrap(); + + let expected_ns = Namespace::with_properties( + NamespaceIdent::new("ns1".to_string()), + HashMap::from([("key1".to_string(), "value1".to_string())]), + ); + + assert_eq!(expected_ns, namespaces); + + config_mock.assert_async().await; + get_ns_mock.assert_async().await; + } + + #[tokio::test] + async fn check_namespace_exists() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let get_ns_mock = server + .mock("HEAD", "/v1/namespaces/ns1") + .with_status(204) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + assert!(catalog + .namespace_exists(&NamespaceIdent::new("ns1".to_string())) + .await + .unwrap()); + + config_mock.assert_async().await; + get_ns_mock.assert_async().await; + } + + #[tokio::test] + async fn test_drop_namespace() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let drop_ns_mock = server + .mock("DELETE", "/v1/namespaces/ns1") + .with_status(204) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + catalog + .drop_namespace(&NamespaceIdent::new("ns1".to_string())) + .await + .unwrap(); + + config_mock.assert_async().await; + drop_ns_mock.assert_async().await; + } + + #[tokio::test] + async fn test_list_tables() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let list_tables_mock = server + .mock("GET", "/v1/namespaces/ns1/tables") + .with_status(200) + .with_body( + r#"{ + "identifiers": [ + { + "namespace": ["ns1"], + "name": "table1" + }, + { + "namespace": ["ns1"], + "name": "table2" + } + ] + }"#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let tables = catalog + .list_tables(&NamespaceIdent::new("ns1".to_string())) + .await + .unwrap(); + + let expected_tables = vec![ + TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table1".to_string()), + TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table2".to_string()), + ]; + + assert_eq!(tables, expected_tables); + + config_mock.assert_async().await; + list_tables_mock.assert_async().await; + } + + #[tokio::test] + async fn test_drop_tables() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let delete_table_mock = server + .mock("DELETE", "/v1/namespaces/ns1/tables/table1") + .with_status(204) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + catalog + .drop_table(&TableIdent::new( + NamespaceIdent::new("ns1".to_string()), + "table1".to_string(), + )) + .await + .unwrap(); + + config_mock.assert_async().await; + delete_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_check_table_exists() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let check_table_exists_mock = server + .mock("HEAD", "/v1/namespaces/ns1/tables/table1") + .with_status(204) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + assert!(catalog + .stat_table(&TableIdent::new( + NamespaceIdent::new("ns1".to_string()), + "table1".to_string(), + )) + .await + .unwrap()); + + config_mock.assert_async().await; + check_table_exists_mock.assert_async().await; + } + + #[tokio::test] + async fn test_rename_table() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let rename_table_mock = server + .mock("POST", "/v1/tables/rename") + .with_status(204) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + catalog + .rename_table( + &TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table1".to_string()), + &TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table2".to_string()), + ) + .await + .unwrap(); + + config_mock.assert_async().await; + rename_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_load_table() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let rename_table_mock = server + .mock("GET", "/v1/namespaces/ns1/tables/test1") + .with_status(200) + .with_body_from_file(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + "load_table_response.json" + )) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let table = catalog + .load_table(&TableIdent::new( + NamespaceIdent::new("ns1".to_string()), + "test1".to_string(), + )) + .await + .unwrap(); + + assert_eq!( + &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), + table.identifier() + ); + assert_eq!("s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json", table.metadata_location().unwrap()); + assert_eq!(FormatVersion::V1, table.metadata().format_version()); + assert_eq!("s3://warehouse/database/table", table.metadata().location()); + assert_eq!( + uuid!("b55d9dda-6561-423a-8bfc-787980ce421f"), + table.metadata().uuid() + ); + assert_eq!( + Utc.timestamp_millis_opt(1646787054459).unwrap(), + table.metadata().last_updated_ms() + ); + assert_eq!( + vec![&Arc::new( + Schema::builder() + .with_fields(vec![ + NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "data", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build() + .unwrap() + )], + table.metadata().schemas_iter().collect::>() + ); + assert_eq!( + &HashMap::from([ + ("owner".to_string(), "bryan".to_string()), + ( + "write.metadata.compression-codec".to_string(), + "gzip".to_string() + ) + ]), + table.metadata().properties() + ); + assert_eq!(vec![&Arc::new(Snapshot::builder() + .with_snapshot_id(3497810964824022504) + .with_timestamp_ms(1646787054459) + .with_manifest_list("s3://warehouse/database/table/metadata/snap-3497810964824022504-1-c4f68204-666b-4e50-a9df-b10c34bf6b82.avro") + .with_sequence_number(0) + .with_schema_id(0) + .with_summary(Summary { + operation: Operation::Append, + other: HashMap::from_iter([ + ("spark.app.id", "local-1646787004168"), + ("added-data-files", "1"), + ("added-records", "1"), + ("added-files-size", "697"), + ("changed-partition-count", "1"), + ("total-records", "1"), + ("total-files-size", "697"), + ("total-data-files", "1"), + ("total-delete-files", "0"), + ("total-position-deletes", "0"), + ("total-equality-deletes", "0") + ].iter().map(|p| (p.0.to_string(), p.1.to_string()))), + }).build() + )], table.metadata().snapshots().collect::>()); + assert_eq!( + &[SnapshotLog { + timestamp_ms: 1646787054459, + snapshot_id: 3497810964824022504, + }], + table.metadata().history() + ); + assert_eq!( + vec![&Arc::new(SortOrder { + order_id: 0, + fields: vec![], + })], + table.metadata().sort_orders_iter().collect::>() + ); + + config_mock.assert_async().await; + rename_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_load_table_404() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let rename_table_mock = server + .mock("GET", "/v1/namespaces/ns1/tables/test1") + .with_status(404) + .with_body(r#" +{ + "error": { + "message": "Table does not exist: ns1.test1 in warehouse 8bcb0838-50fc-472d-9ddb-8feb89ef5f1e", + "type": "NoSuchNamespaceErrorException", + "code": 404 + } +} + "#) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let table = catalog + .load_table(&TableIdent::new( + NamespaceIdent::new("ns1".to_string()), + "test1".to_string(), + )) + .await; + + assert!(table.is_err()); + assert!(table + .err() + .unwrap() + .message() + .contains("Table does not exist")); + + config_mock.assert_async().await; + rename_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_create_table() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let create_table_mock = server + .mock("POST", "/v1/namespaces/ns1/tables") + .with_status(200) + .with_body_from_file(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + "create_table_response.json" + )) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .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(), + ) + .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) + .partition_spec( + UnboundPartitionSpec::builder() + .with_fields(vec![UnboundPartitionField::builder() + .source_id(1) + .transform(Transform::Truncate(3)) + .name("id".to_string()) + .build()]) + .build() + .unwrap(), + ) + .sort_order( + SortOrder::builder() + .with_sort_field( + SortField::builder() + .source_id(2) + .transform(Transform::Identity) + .direction(SortDirection::Ascending) + .null_order(NullOrder::First) + .build(), + ) + .build_unbound() + .unwrap(), + ) + .build(); + + let table = catalog + .create_table(&NamespaceIdent::from_strs(["ns1"]).unwrap(), table_creation) + .await + .unwrap(); + + assert_eq!( + &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), + table.identifier() + ); + assert_eq!( + "s3://warehouse/database/table/metadata.json", + table.metadata_location().unwrap() + ); + assert_eq!(FormatVersion::V1, table.metadata().format_version()); + assert_eq!("s3://warehouse/database/table", table.metadata().location()); + assert_eq!( + uuid!("bf289591-dcc0-4234-ad4f-5c3eed811a29"), + table.metadata().uuid() + ); + assert_eq!( + 1657810967051, + table.metadata().last_updated_ms().timestamp_millis() + ); + assert_eq!( + vec![&Arc::new( + 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(0) + .with_identifier_field_ids(vec![2]) + .build() + .unwrap() + )], + table.metadata().schemas_iter().collect::>() + ); + assert_eq!( + &HashMap::from([ + ( + "write.delete.parquet.compression-codec".to_string(), + "zstd".to_string() + ), + ( + "write.metadata.compression-codec".to_string(), + "gzip".to_string() + ), + ( + "write.summary.partition-limit".to_string(), + "100".to_string() + ), + ( + "write.parquet.compression-codec".to_string(), + "zstd".to_string() + ), + ]), + table.metadata().properties() + ); + assert!(table.metadata().current_snapshot().is_none()); + assert!(table.metadata().history().is_empty()); + assert_eq!( + vec![&Arc::new(SortOrder { + order_id: 0, + fields: vec![], + })], + table.metadata().sort_orders_iter().collect::>() + ); + + config_mock.assert_async().await; + create_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_create_table_409() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let create_table_mock = server + .mock("POST", "/v1/namespaces/ns1/tables") + .with_status(409) + .with_body(r#" +{ + "error": { + "message": "Table already exists: ns1.test1 in warehouse 8bcb0838-50fc-472d-9ddb-8feb89ef5f1e", + "type": "AlreadyExistsException", + "code": 409 + } +} + "#) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .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(), + ) + .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) + .build(); + + let table_result = catalog + .create_table(&NamespaceIdent::from_strs(["ns1"]).unwrap(), table_creation) + .await; + + assert!(table_result.is_err()); + assert!(table_result + .err() + .unwrap() + .message() + .contains("Table already exists")); + + config_mock.assert_async().await; + create_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_update_table() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let update_table_mock = server + .mock("POST", "/v1/namespaces/ns1/tables/test1") + .with_status(200) + .with_body_from_file(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + "update_table_response.json" + )) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let table1 = { + let file = File::open(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + "create_table_response.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, LoadTableResponse>(reader).unwrap(); + + Table::builder() + .metadata(resp.metadata) + .metadata_location(resp.metadata_location.unwrap()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .build() + }; + + let table = Transaction::new(&table1) + .upgrade_table_version(FormatVersion::V2) + .unwrap() + .commit(&catalog) + .await + .unwrap(); + + assert_eq!( + &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), + table.identifier() + ); + assert_eq!( + "s3://warehouse/database/table/metadata.json", + table.metadata_location().unwrap() + ); + assert_eq!(FormatVersion::V2, table.metadata().format_version()); + assert_eq!("s3://warehouse/database/table", table.metadata().location()); + assert_eq!( + uuid!("bf289591-dcc0-4234-ad4f-5c3eed811a29"), + table.metadata().uuid() + ); + assert_eq!( + 1657810967051, + table.metadata().last_updated_ms().timestamp_millis() + ); + assert_eq!( + vec![&Arc::new( + 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(0) + .with_identifier_field_ids(vec![2]) + .build() + .unwrap() + )], + table.metadata().schemas_iter().collect::>() + ); + assert_eq!( + &HashMap::from([ + ( + "write.delete.parquet.compression-codec".to_string(), + "zstd".to_string() + ), + ( + "write.metadata.compression-codec".to_string(), + "gzip".to_string() + ), + ( + "write.summary.partition-limit".to_string(), + "100".to_string() + ), + ( + "write.parquet.compression-codec".to_string(), + "zstd".to_string() + ), + ]), + table.metadata().properties() + ); + assert!(table.metadata().current_snapshot().is_none()); + assert!(table.metadata().history().is_empty()); + assert_eq!( + vec![&Arc::new(SortOrder { + order_id: 0, + fields: vec![], + })], + table.metadata().sort_orders_iter().collect::>() + ); + + config_mock.assert_async().await; + update_table_mock.assert_async().await; + } + + #[tokio::test] + async fn test_update_table_404() { + let mut server = Server::new_async().await; + + let config_mock = create_config_mock(&mut server).await; + + let update_table_mock = server + .mock("POST", "/v1/namespaces/ns1/tables/test1") + .with_status(404) + .with_body( + r#" +{ + "error": { + "message": "The given table does not exist", + "type": "NoSuchTableException", + "code": 404 + } +} + "#, + ) + .create_async() + .await; + + let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) + .await + .unwrap(); + + let table1 = { + let file = File::open(format!( + "{}/testdata/{}", + env!("CARGO_MANIFEST_DIR"), + "create_table_response.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, LoadTableResponse>(reader).unwrap(); + + Table::builder() + .metadata(resp.metadata) + .metadata_location(resp.metadata_location.unwrap()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .build() + }; + + let table_result = Transaction::new(&table1) + .upgrade_table_version(FormatVersion::V2) + .unwrap() + .commit(&catalog) + .await; + + assert!(table_result.is_err()); + assert!(table_result + .err() + .unwrap() + .message() + .contains("The given table does not exist")); + + config_mock.assert_async().await; + update_table_mock.assert_async().await; + } +} diff --git a/crates/rest/src/lib.rs b/crates/rest/src/lib.rs new file mode 100644 index 0000000..023fe7a --- /dev/null +++ b/crates/rest/src/lib.rs @@ -0,0 +1,23 @@ +// 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. + +//! Iceberg REST API implementation. + +#![deny(missing_docs)] + +mod catalog; +pub use catalog::*; diff --git a/crates/rest/testdata/create_table_response.json b/crates/rest/testdata/create_table_response.json new file mode 100644 index 0000000..e01a52f --- /dev/null +++ b/crates/rest/testdata/create_table_response.json @@ -0,0 +1,53 @@ +{ + "metadata-location": "s3://warehouse/database/table/metadata.json", + "metadata": { + "format-version": 1, + "table-uuid": "bf289591-dcc0-4234-ad4f-5c3eed811a29", + "location": "s3://warehouse/database/table", + "last-updated-ms": 1657810967051, + "last-column-id": 3, + "schema": { + "type": "struct", + "schema-id": 0, + "identifier-field-ids": [2], + "fields": [ + {"id": 1, "name": "foo", "required": false, "type": "string"}, + {"id": 2, "name": "bar", "required": true, "type": "int"}, + {"id": 3, "name": "baz", "required": false, "type": "boolean"} + ] + }, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "identifier-field-ids": [2], + "fields": [ + {"id": 1, "name": "foo", "required": false, "type": "string"}, + {"id": 2, "name": "bar", "required": true, "type": "int"}, + {"id": 3, "name": "baz", "required": false, "type": "boolean"} + ] + } + ], + "partition-spec": [], + "default-spec-id": 0, + "last-partition-id": 999, + "default-sort-order-id": 0, + "sort-orders": [{"order-id": 0, "fields": []}], + "properties": { + "write.delete.parquet.compression-codec": "zstd", + "write.metadata.compression-codec": "gzip", + "write.summary.partition-limit": "100", + "write.parquet.compression-codec": "zstd" + }, + "current-snapshot-id": -1, + "refs": {}, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] + }, + "config": { + "client.factory": "io.tabular.iceberg.catalog.TabularAwsClientFactory", + "region": "us-west-2" + } +} \ No newline at end of file diff --git a/crates/rest/testdata/load_table_response.json b/crates/rest/testdata/load_table_response.json new file mode 100644 index 0000000..012f0e9 --- /dev/null +++ b/crates/rest/testdata/load_table_response.json @@ -0,0 +1,68 @@ +{ + "metadata-location": "s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json", + "metadata": { + "format-version": 1, + "table-uuid": "b55d9dda-6561-423a-8bfc-787980ce421f", + "location": "s3://warehouse/database/table", + "last-updated-ms": 1646787054459, + "last-column-id": 2, + "schema": { + "type": "struct", + "schema-id": 0, + "fields": [ + {"id": 1, "name": "id", "required": false, "type": "int"}, + {"id": 2, "name": "data", "required": false, "type": "string"} + ] + }, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + {"id": 1, "name": "id", "required": false, "type": "int"}, + {"id": 2, "name": "data", "required": false, "type": "string"} + ] + } + ], + "partition-spec": [], + "default-spec-id": 0, + "partition-specs": [{"spec-id": 0, "fields": []}], + "last-partition-id": 999, + "default-sort-order-id": 0, + "sort-orders": [{"order-id": 0, "fields": []}], + "properties": {"owner": "bryan", "write.metadata.compression-codec": "gzip"}, + "current-snapshot-id": 3497810964824022504, + "refs": {"main": {"snapshot-id": 3497810964824022504, "type": "branch"}}, + "snapshots": [ + { + "snapshot-id": 3497810964824022504, + "timestamp-ms": 1646787054459, + "summary": { + "operation": "append", + "spark.app.id": "local-1646787004168", + "added-data-files": "1", + "added-records": "1", + "added-files-size": "697", + "changed-partition-count": "1", + "total-records": "1", + "total-files-size": "697", + "total-data-files": "1", + "total-delete-files": "0", + "total-position-deletes": "0", + "total-equality-deletes": "0" + }, + "manifest-list": "s3://warehouse/database/table/metadata/snap-3497810964824022504-1-c4f68204-666b-4e50-a9df-b10c34bf6b82.avro", + "schema-id": 0 + } + ], + "snapshot-log": [{"timestamp-ms": 1646787054459, "snapshot-id": 3497810964824022504}], + "metadata-log": [ + { + "timestamp-ms": 1646787031514, + "metadata-file": "s3://warehouse/database/table/metadata/00000-88484a1c-00e5-4a07-a787-c0e7aeffa805.gz.metadata.json" + } + ] + }, + "config": {"client.factory": "io.tabular.iceberg.catalog.TabularAwsClientFactory", "region": "us-west-2"} +} \ No newline at end of file diff --git a/crates/rest/testdata/rest_catalog/docker-compose.yaml b/crates/rest/testdata/rest_catalog/docker-compose.yaml new file mode 100644 index 0000000..5c10146 --- /dev/null +++ b/crates/rest/testdata/rest_catalog/docker-compose.yaml @@ -0,0 +1,65 @@ +# 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. + +version: '3.8' + +services: + rest: + image: tabulario/iceberg-rest:0.10.0 + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + - CATALOG_CATOLOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory + - CATALOG_WAREHOUSE=s3://icebergdata/demo + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + depends_on: + - minio + links: + - minio:icebergdata.minio + expose: + - 8181 + + minio: + image: minio/minio + environment: + - MINIO_ROOT_USER=admin + - MINIO_ROOT_PASSWORD=password + - MINIO_DOMAIN=minio + expose: + - 9001 + - 9000 + command: [ "server", "/data", "--console-address", ":9001" ] + + mc: + depends_on: + - minio + image: minio/mc + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " + until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done; + /usr/bin/mc rm -r --force minio/icebergdata; + /usr/bin/mc mb minio/icebergdata; + /usr/bin/mc policy set public minio/icebergdata; + tail -f /dev/null + " \ No newline at end of file diff --git a/crates/rest/testdata/update_table_response.json b/crates/rest/testdata/update_table_response.json new file mode 100644 index 0000000..80ec269 --- /dev/null +++ b/crates/rest/testdata/update_table_response.json @@ -0,0 +1,40 @@ +{ + "metadata-location": "s3://warehouse/database/table/metadata.json", + "metadata": { + "format-version": 2, + "table-uuid": "bf289591-dcc0-4234-ad4f-5c3eed811a29", + "location": "s3://warehouse/database/table", + "last-sequence-number" : 1, + "last-updated-ms": 1657810967051, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "identifier-field-ids": [2], + "fields": [ + {"id": 1, "name": "foo", "required": false, "type": "string"}, + {"id": 2, "name": "bar", "required": true, "type": "int"}, + {"id": 3, "name": "baz", "required": false, "type": "boolean"} + ] + } + ], + "partition-specs": [], + "default-spec-id": 0, + "last-partition-id": 999, + "default-sort-order-id": 0, + "sort-orders": [{"order-id": 0, "fields": []}], + "properties": { + "write.delete.parquet.compression-codec": "zstd", + "write.metadata.compression-codec": "gzip", + "write.summary.partition-limit": "100", + "write.parquet.compression-codec": "zstd" + }, + "current-snapshot-id": -1, + "refs": {}, + "snapshots": [], + "snapshot-log": [], + "metadata-log": [] + } +} \ No newline at end of file diff --git a/crates/rest/tests/rest_catalog_test.rs b/crates/rest/tests/rest_catalog_test.rs new file mode 100644 index 0000000..a4d0795 --- /dev/null +++ b/crates/rest/tests/rest_catalog_test.rs @@ -0,0 +1,376 @@ +// 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. + +//! Integration tests for rest catalog. + +use iceberg::spec::{FormatVersion, NestedField, PrimitiveType, Schema, Type}; +use iceberg::transaction::Transaction; +use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; +use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; +use iceberg_test_utils::docker::DockerCompose; +use iceberg_test_utils::{normalize_test_name, set_up}; +use port_scanner::scan_port_addr; +use std::collections::HashMap; +use tokio::time::sleep; + +const REST_CATALOG_PORT: u16 = 8181; + +struct TestFixture { + _docker_compose: DockerCompose, + rest_catalog: RestCatalog, +} + +async fn set_test_fixture(func: &str) -> TestFixture { + set_up(); + let docker_compose = DockerCompose::new( + normalize_test_name(format!("{}_{func}", module_path!())), + format!("{}/testdata/rest_catalog", env!("CARGO_MANIFEST_DIR")), + ); + + // Start docker compose + docker_compose.run(); + + let rest_catalog_ip = docker_compose.get_container_ip("rest"); + + let read_port = format!("{}:{}", rest_catalog_ip, REST_CATALOG_PORT); + loop { + if !scan_port_addr(&read_port) { + log::info!("Waiting for 1s rest catalog to ready..."); + sleep(std::time::Duration::from_millis(1000)).await; + } else { + break; + } + } + + let config = RestCatalogConfig::builder() + .uri(format!("http://{}:{}", rest_catalog_ip, REST_CATALOG_PORT)) + .build(); + let rest_catalog = RestCatalog::new(config).await.unwrap(); + + TestFixture { + _docker_compose: docker_compose, + rest_catalog, + } +} +#[tokio::test] +async fn test_get_non_exist_namespace() { + let fixture = set_test_fixture("test_get_non_exist_namespace").await; + + let result = fixture + .rest_catalog + .get_namespace(&NamespaceIdent::from_strs(["demo"]).unwrap()) + .await; + + assert!(result.is_err()); + assert!(result + .unwrap_err() + .to_string() + .contains("Namespace does not exist")); +} + +#[tokio::test] +async fn test_get_namespace() { + let fixture = set_test_fixture("test_get_namespace").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Verify that namespace doesn't exist + assert!(fixture.rest_catalog.get_namespace(ns.name()).await.is_err()); + + // Create this namespace + let created_ns = fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + assert_eq!(ns.name(), created_ns.name()); + assert_map_contains(ns.properties(), created_ns.properties()); + + // Check that this namespace already exists + let get_ns = fixture.rest_catalog.get_namespace(ns.name()).await.unwrap(); + assert_eq!(ns.name(), get_ns.name()); + assert_map_contains(ns.properties(), created_ns.properties()); +} + +#[tokio::test] +async fn test_list_namespace() { + let fixture = set_test_fixture("test_list_namespace").await; + + let ns1 = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + let ns2 = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "macos"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "xuanwo".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Currently this namespace doesn't exist, so it should return error. + assert!(fixture + .rest_catalog + .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) + .await + .is_err()); + + // Create namespaces + fixture + .rest_catalog + .create_namespace(ns1.name(), ns1.properties().clone()) + .await + .unwrap(); + fixture + .rest_catalog + .create_namespace(ns2.name(), ns1.properties().clone()) + .await + .unwrap(); + + // List namespace + let mut nss = fixture + .rest_catalog + .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) + .await + .unwrap(); + nss.sort(); + + assert_eq!(&nss[0], ns1.name()); + assert_eq!(&nss[1], ns2.name()); +} + +#[tokio::test] +async fn test_list_empty_namespace() { + let fixture = set_test_fixture("test_list_empty_namespace").await; + + let ns_apple = Namespace::with_properties( + NamespaceIdent::from_strs(["apple"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Currently this namespace doesn't exist, so it should return error. + assert!(fixture + .rest_catalog + .list_namespaces(Some(ns_apple.name())) + .await + .is_err()); + + // Create namespaces + fixture + .rest_catalog + .create_namespace(ns_apple.name(), ns_apple.properties().clone()) + .await + .unwrap(); + + // List namespace + let nss = fixture + .rest_catalog + .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) + .await + .unwrap(); + assert!(nss.is_empty()); +} + +#[tokio::test] +async fn test_list_root_namespace() { + let fixture = set_test_fixture("test_list_root_namespace").await; + + let ns1 = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + let ns2 = Namespace::with_properties( + NamespaceIdent::from_strs(["google", "android"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "xuanwo".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Currently this namespace doesn't exist, so it should return error. + assert!(fixture + .rest_catalog + .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) + .await + .is_err()); + + // Create namespaces + fixture + .rest_catalog + .create_namespace(ns1.name(), ns1.properties().clone()) + .await + .unwrap(); + fixture + .rest_catalog + .create_namespace(ns2.name(), ns1.properties().clone()) + .await + .unwrap(); + + // List namespace + let mut nss = fixture.rest_catalog.list_namespaces(None).await.unwrap(); + nss.sort(); + + assert_eq!(&nss[0], &NamespaceIdent::from_strs(["apple"]).unwrap()); + assert_eq!(&nss[1], &NamespaceIdent::from_strs(["google"]).unwrap()); +} + +#[tokio::test] +async fn test_create_table() { + let fixture = set_test_fixture("test_create_table").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Create namespaces + fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .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(), + ]) + .build() + .unwrap(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = fixture + .rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + assert_eq!( + table.identifier(), + &TableIdent::new(ns.name().clone(), "t1".to_string()) + ); + + assert_eq!( + table.metadata().current_schema().as_struct(), + schema.as_struct() + ); + assert_eq!(table.metadata().format_version(), FormatVersion::V2); + assert!(table.metadata().current_snapshot().is_none()); + assert!(table.metadata().history().is_empty()); + assert!(table.metadata().default_sort_order().unwrap().is_unsorted()); + assert!(table + .metadata() + .default_partition_spec() + .unwrap() + .is_unpartitioned()); +} + +#[tokio::test] +async fn test_update_table() { + let fixture = set_test_fixture("test_update_table").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + // Create namespaces + fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .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(), + ]) + .build() + .unwrap(); + + // Now we create a table + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = fixture + .rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + assert_eq!( + table.identifier(), + &TableIdent::new(ns.name().clone(), "t1".to_string()) + ); + + // Update table by committing transaction + let table2 = Transaction::new(&table) + .set_properties(HashMap::from([("prop1".to_string(), "v1".to_string())])) + .unwrap() + .commit(&fixture.rest_catalog) + .await + .unwrap(); + + assert_map_contains( + &HashMap::from([("prop1".to_string(), "v1".to_string())]), + table2.metadata().properties(), + ); +} + +fn assert_map_contains(map1: &HashMap, map2: &HashMap) { + for (k, v) in map1 { + assert!(map2.contains_key(k)); + assert_eq!(map2.get(k).unwrap(), v); + } +} diff --git a/crates/test_utils/Cargo.toml b/crates/test_utils/Cargo.toml new file mode 100644 index 0000000..2ad7ef9 --- /dev/null +++ b/crates/test_utils/Cargo.toml @@ -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. + +[package] +name = "iceberg_test_utils" +version = { workspace = true } +edition = { workspace = true } +rust-version = { workspace = true } +repository = { workspace = true } +license = { workspace = true } + +[dependencies] +env_logger = { workspace = true } +log = "0.4.20" + +[features] +tests = [] diff --git a/crates/test_utils/src/cmd.rs b/crates/test_utils/src/cmd.rs new file mode 100644 index 0000000..604d4a1 --- /dev/null +++ b/crates/test_utils/src/cmd.rs @@ -0,0 +1,41 @@ +// 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::process::Command; + +pub fn run_command(mut cmd: Command, desc: impl ToString) { + let desc = desc.to_string(); + log::info!("Starting to {}, command: {:?}", &desc, cmd); + let exit = cmd.status().unwrap(); + if exit.success() { + log::info!("{} succeed!", desc) + } else { + panic!("{} failed: {:?}", desc, exit); + } +} + +pub fn get_cmd_output(mut cmd: Command, desc: impl ToString) -> String { + let desc = desc.to_string(); + log::info!("Starting to {}, command: {:?}", &desc, cmd); + let output = cmd.output().unwrap(); + if output.status.success() { + log::info!("{} succeed!", desc); + String::from_utf8(output.stdout).unwrap() + } else { + panic!("{} failed: {:?}", desc, output.status); + } +} diff --git a/crates/test_utils/src/docker.rs b/crates/test_utils/src/docker.rs new file mode 100644 index 0000000..6c5fbef --- /dev/null +++ b/crates/test_utils/src/docker.rs @@ -0,0 +1,102 @@ +// 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 crate::cmd::{get_cmd_output, run_command}; +use std::process::Command; + +/// A utility to manage lifecycle of docker compose. +/// +/// It's will start docker compose when calling `run` method, and will be stopped when dropped. +#[derive(Debug)] +pub struct DockerCompose { + project_name: String, + docker_compose_dir: String, +} + +impl DockerCompose { + pub fn new(project_name: impl ToString, docker_compose_dir: impl ToString) -> Self { + Self { + project_name: project_name.to_string(), + docker_compose_dir: docker_compose_dir.to_string(), + } + } + + pub fn project_name(&self) -> &str { + self.project_name.as_str() + } + + pub fn run(&self) { + let mut cmd = Command::new("docker"); + cmd.current_dir(&self.docker_compose_dir); + + cmd.args(vec![ + "compose", + "-p", + self.project_name.as_str(), + "up", + "-d", + "--wait", + "--timeout", + "1200000", + ]); + + run_command( + cmd, + format!( + "Starting docker compose in {}, project name: {}", + self.docker_compose_dir, self.project_name + ), + ) + } + + pub fn get_container_ip(&self, service_name: impl AsRef) -> String { + let container_name = format!("{}-{}-1", self.project_name, service_name.as_ref()); + let mut cmd = Command::new("docker"); + cmd.arg("inspect") + .arg("-f") + .arg("{{range.NetworkSettings.Networks}}{{.IPAddress}}{{end}}") + .arg(&container_name); + + get_cmd_output(cmd, format!("Get container ip of {container_name}")) + .trim() + .to_string() + } +} + +impl Drop for DockerCompose { + fn drop(&mut self) { + let mut cmd = Command::new("docker"); + cmd.current_dir(&self.docker_compose_dir); + + cmd.args(vec![ + "compose", + "-p", + self.project_name.as_str(), + "down", + "-v", + "--remove-orphans", + ]); + + run_command( + cmd, + format!( + "Stopping docker compose in {}, project name: {}", + self.docker_compose_dir, self.project_name + ), + ) + } +} diff --git a/crates/test_utils/src/lib.rs b/crates/test_utils/src/lib.rs new file mode 100644 index 0000000..4f63b8d --- /dev/null +++ b/crates/test_utils/src/lib.rs @@ -0,0 +1,41 @@ +// 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. + +//! This crate contains common utilities for testing. +//! +//! It's not intended for use outside of `iceberg-rust`. + +#[cfg(feature = "tests")] +mod cmd; +#[cfg(feature = "tests")] +pub mod docker; + +#[cfg(feature = "tests")] +pub use common::*; + +#[cfg(feature = "tests")] +mod common { + use std::sync::Once; + + static INIT: Once = Once::new(); + pub fn set_up() { + INIT.call_once(env_logger::init); + } + pub fn normalize_test_name(s: impl ToString) -> String { + s.to_string().replace("::", "__").replace('.', "_") + } +} From af468fd760f4d668ca2b5d3d3faf2e98108cd305 Mon Sep 17 00:00:00 2001 From: zhouzilong <529620861@qq.com> Date: Tue, 27 Feb 2024 09:33:24 -0500 Subject: [PATCH 2/4] server struct --- Cargo.toml | 59 +- Makefile | 4 + crates/rest/Cargo.toml | 48 - crates/rest/DEPENDENCIES.rust.tsv | 295 --- crates/rest/src/catalog.rs | 1604 ----------------- crates/rest/src/lib.rs | 23 - .../rest/testdata/create_table_response.json | 53 - crates/rest/testdata/load_table_response.json | 68 - .../testdata/rest_catalog/docker-compose.yaml | 65 - .../rest/testdata/update_table_response.json | 40 - crates/rest/tests/rest_catalog_test.rs | 376 ---- crates/test_utils/Cargo.toml | 31 - crates/test_utils/src/cmd.rs | 41 - crates/test_utils/src/docker.rs | 102 -- crates/test_utils/src/lib.rs | 41 - rustfmt.toml | 4 + scripts/parse_dependencies.py | 42 + src/main.rs | 31 +- src/server/mod.rs | 1 + src/server/routes/config.rs | 4 + src/server/routes/metric.rs | 5 + src/server/routes/mod.rs | 5 + src/server/routes/namespace.rs | 35 + src/server/routes/table.rs | 47 + 24 files changed, 185 insertions(+), 2839 deletions(-) delete mode 100644 crates/rest/Cargo.toml delete mode 100644 crates/rest/DEPENDENCIES.rust.tsv delete mode 100644 crates/rest/src/catalog.rs delete mode 100644 crates/rest/src/lib.rs delete mode 100644 crates/rest/testdata/create_table_response.json delete mode 100644 crates/rest/testdata/load_table_response.json delete mode 100644 crates/rest/testdata/rest_catalog/docker-compose.yaml delete mode 100644 crates/rest/testdata/update_table_response.json delete mode 100644 crates/rest/tests/rest_catalog_test.rs delete mode 100644 crates/test_utils/Cargo.toml delete mode 100644 crates/test_utils/src/cmd.rs delete mode 100644 crates/test_utils/src/docker.rs delete mode 100644 crates/test_utils/src/lib.rs create mode 100644 rustfmt.toml create mode 100644 scripts/parse_dependencies.py create mode 100644 src/server/mod.rs create mode 100644 src/server/routes/config.rs create mode 100644 src/server/routes/metric.rs create mode 100644 src/server/routes/mod.rs create mode 100644 src/server/routes/namespace.rs create mode 100644 src/server/routes/table.rs diff --git a/Cargo.toml b/Cargo.toml index 282d0a2..67d32d9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -15,58 +15,17 @@ # specific language governing permissions and limitations # under the License. -[workspace] -resolver = "2" -members = ["crates/rest", "crates/examples", "crates/iceberg", "crates/test_utils"] - -[workspace.package] -version = "0.2.0" +[package] +name = "catalog2" +version = "0.1.0" edition = "2021" - license = "Apache-2.0" repository = "https://github.com/cmu-db/15721-s24-catalog2" rust-version = "1.75.0" -[workspace.dependencies] -anyhow = "1.0.72" -apache-avro = "0.16" -arrow-arith = { version = ">=46" } -arrow-array = { version = ">=46" } -arrow-schema = { version = ">=46" } -async-trait = "0.1" -bimap = "0.6" -bitvec = "1.0.1" -chrono = "0.4" -derive_builder = "0.13.0" -either = "1" -env_logger = "0.11.0" -futures = "0.3" -iceberg = { path = "./crates/iceberg" } -iceberg-catalog-rest = { path = "./crates/rest" } -itertools = "0.12" -lazy_static = "1" -log = "^0.4" -mockito = "^1" -murmur3 = "0.5.2" -once_cell = "1" -opendal = "0.45" -ordered-float = "4.0.0" -pretty_assertions = "1.4.0" -port_scanner = "0.1.5" -reqwest = { version = "^0.11", features = ["json"] } -rust_decimal = "1.31.0" -serde = { version = "^1.0", features = ["rc"] } -serde_bytes = "0.11.8" -serde_derive = "^1.0" -serde_json = "^1.0" -serde_repr = "0.1.16" -serde_with = "3.4.0" -tempfile = "3.8" -tokio = { version = "1", features = ["macros"] } -typed-builder = "^0.18" -url = "2" -urlencoding = "2" -uuid = "1.6.1" -volo-thrift = "0.9.2" -hive_metastore = "0.0.2" -tera = "1" + + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html +[dependencies] +rocket = { version = "0.5.0", features = ["json", "http2"] } +dotenv = "0.15.0" \ No newline at end of file diff --git a/Makefile b/Makefile index d411cce..efc7fd7 100644 --- a/Makefile +++ b/Makefile @@ -22,6 +22,10 @@ RUST_LOG = debug build: cargo build +run: + cargo build + target/debug/catalog2 + check-fmt: cargo fmt --all -- --check diff --git a/crates/rest/Cargo.toml b/crates/rest/Cargo.toml deleted file mode 100644 index 6db3060..0000000 --- a/crates/rest/Cargo.toml +++ /dev/null @@ -1,48 +0,0 @@ -# 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-rest" -version = { workspace = true } -edition = { workspace = true } -rust-version = { workspace = true } - -categories = ["database"] -description = "Apache Iceberg Rust REST API" -repository = { workspace = true } -license = { workspace = true } -keywords = ["iceberg", "rest", "catalog"] - -[dependencies] -# async-trait = { workspace = true } -async-trait = { workspace = true } -chrono = { workspace = true } -iceberg = { workspace = true } -log = "0.4.20" -reqwest = { workspace = true } -serde = { workspace = true } -serde_derive = { workspace = true } -serde_json = { workspace = true } -typed-builder = { workspace = true } -urlencoding = { workspace = true } -uuid = { workspace = true, features = ["v4"] } - -[dev-dependencies] -iceberg_test_utils = { path = "../test_utils", features = ["tests"] } -mockito = { workspace = true } -port_scanner = { workspace = true } -tokio = { workspace = true } diff --git a/crates/rest/DEPENDENCIES.rust.tsv b/crates/rest/DEPENDENCIES.rust.tsv deleted file mode 100644 index 6061ce8..0000000 --- a/crates/rest/DEPENDENCIES.rust.tsv +++ /dev/null @@ -1,295 +0,0 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.21.0 X X -adler@1.0.2 X X X -adler32@1.2.0 X -ahash@0.8.6 X X -aho-corasick@1.1.2 X X -android-tzdata@0.1.1 X X -android_system_properties@0.1.5 X X -anstream@0.6.11 X X -anstyle@1.0.4 X X -anstyle-parse@0.2.3 X X -anstyle-query@1.0.2 X X -anstyle-wincon@3.0.2 X X -anyhow@1.0.77 X X -apache-avro@0.16.0 X -arrayvec@0.7.4 X X -arrow-arith@49.0.0 X -arrow-array@49.0.0 X -arrow-buffer@49.0.0 X -arrow-data@49.0.0 X -arrow-schema@49.0.0 X -async-compat@0.2.3 X X -async-trait@0.1.75 X X -autocfg@1.1.0 X X -backon@0.4.1 X -backtrace@0.3.69 X X -base64@0.21.5 X X -base64ct@1.6.0 X X -bimap@0.6.3 X X -bitflags@1.3.2 X X -bitflags@2.4.1 X X -bitvec@1.0.1 X -block-buffer@0.10.4 X X -bumpalo@3.14.0 X X -byteorder@1.5.0 X X -bytes@1.5.0 X -cc@1.0.83 X X -cfg-if@1.0.0 X X -chrono@0.4.31 X X -colorchoice@1.0.0 X X -const-oid@0.9.6 X X -const-random@0.1.17 X X -const-random-macro@0.1.16 X X -core-foundation@0.9.4 X X -core-foundation-sys@0.8.6 X X -core2@0.4.0 X X -cpufeatures@0.2.11 X X -crc32fast@1.3.2 X X -crunchy@0.2.2 X -crypto-common@0.1.6 X X -darling@0.14.4 X -darling@0.20.3 X -darling_core@0.14.4 X -darling_core@0.20.3 X -darling_macro@0.14.4 X -darling_macro@0.20.3 X -dary_heap@0.3.6 X X -der@0.7.8 X X -deranged@0.3.10 X X -derive_builder@0.13.0 X X -derive_builder_core@0.13.0 X X -derive_builder_macro@0.13.0 X X -digest@0.10.7 X X -dlv-list@0.5.2 X X -either@1.9.0 X X -encoding_rs@0.8.33 X X X -env_filter@0.1.0 X X -env_logger@0.11.0 X X -equivalent@1.0.1 X X -fastrand@1.9.0 X X -fastrand@2.0.1 X X -flagset@0.4.4 X -fnv@1.0.7 X X -foreign-types@0.3.2 X X -foreign-types-shared@0.1.1 X X -form_urlencoded@1.2.1 X X -funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X -generic-array@0.14.7 X -getrandom@0.2.11 X X -gimli@0.28.1 X X -h2@0.3.22 X -half@2.3.1 X X -hashbrown@0.13.2 X X -hashbrown@0.14.3 X X -heck@0.4.1 X X -hermit-abi@0.3.3 X X -hex@0.4.3 X X -hmac@0.12.1 X X -home@0.5.9 X X -http@0.2.11 X X -http-body@0.4.6 X -httparse@1.8.0 X X -httpdate@1.0.3 X X -humantime@2.1.0 X X -hyper@0.14.28 X -hyper-rustls@0.24.2 X X X -hyper-tls@0.5.0 X X -iana-time-zone@0.1.58 X X -iana-time-zone-haiku@0.1.2 X X -iceberg@0.2.0 X -iceberg-catalog-rest@0.2.0 X -iceberg_test_utils@0.2.0 X -ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.1.0 X X -instant@0.1.12 X -ipnet@2.9.0 X X -itertools@0.12.0 X X -itoa@1.0.10 X X -js-sys@0.3.66 X X -jsonwebtoken@9.2.0 X -lazy_static@1.4.0 X X -libc@0.2.151 X X -libflate@2.0.0 X -libflate_lz77@2.0.0 X -libm@0.2.8 X X -linux-raw-sys@0.4.12 X X X -lock_api@0.4.11 X X -log@0.4.20 X X -md-5@0.10.6 X X -memchr@2.6.4 X X -mime@0.3.17 X X -miniz_oxide@0.7.1 X X X -mio@0.8.10 X -murmur3@0.5.2 X X -native-tls@0.2.11 X X -num@0.4.1 X X -num-bigint@0.4.4 X X -num-bigint-dig@0.8.4 X X -num-complex@0.4.4 X X -num-integer@0.1.45 X X -num-iter@0.1.43 X X -num-rational@0.4.1 X X -num-traits@0.2.17 X X -num_cpus@1.16.0 X X -object@0.32.2 X X -once_cell@1.19.0 X X -opendal@0.44.0 X -openssl@0.10.62 X -openssl-macros@0.1.1 X X -openssl-probe@0.1.5 X X -openssl-sys@0.9.98 X -ordered-float@4.2.0 X -ordered-multimap@0.7.1 X -parking_lot@0.12.1 X X -parking_lot_core@0.9.9 X X -pem@3.0.3 X -pem-rfc7468@0.7.0 X X -percent-encoding@2.3.1 X X -pin-project@1.1.3 X X -pin-project-internal@1.1.3 X X -pin-project-lite@0.2.13 X X -pin-utils@0.1.0 X X -pkcs1@0.7.5 X X -pkcs8@0.10.2 X X -pkg-config@0.3.28 X X -powerfmt@0.2.0 X X -ppv-lite86@0.2.17 X X -proc-macro2@1.0.71 X X -quad-rand@0.2.1 X -quick-xml@0.30.0 X -quick-xml@0.31.0 X -quote@1.0.33 X X -radium@0.7.0 X -rand@0.8.5 X X -rand_chacha@0.3.1 X X -rand_core@0.6.4 X X -redox_syscall@0.4.1 X -regex@1.10.2 X X -regex-automata@0.4.3 X X -regex-lite@0.1.5 X X -regex-syntax@0.8.2 X X -reqsign@0.14.6 X -reqwest@0.11.23 X X -ring@0.17.7 X -rle-decode-fast@1.0.3 X X -rsa@0.9.6 X X -rust-ini@0.20.0 X -rust_decimal@1.33.1 X -rustc-demangle@0.1.23 X X -rustix@0.38.28 X X X -rustls@0.21.10 X X X -rustls-native-certs@0.6.3 X X X -rustls-pemfile@1.0.4 X X X -rustls-webpki@0.101.7 X -rustversion@1.0.14 X X -ryu@1.0.16 X X -schannel@0.1.22 X -scopeguard@1.2.0 X X -sct@0.7.1 X X X -security-framework@2.9.2 X X -security-framework-sys@2.9.1 X X -serde@1.0.193 X X -serde_bytes@0.11.13 X X -serde_derive@1.0.193 X X -serde_json@1.0.108 X X -serde_repr@0.1.17 X X -serde_urlencoded@0.7.1 X X -serde_with@3.4.0 X X -serde_with_macros@3.4.0 X X -sha1@0.10.6 X X -sha2@0.10.8 X X -signal-hook-registry@1.4.1 X X -signature@2.2.0 X X -simple_asn1@0.6.2 X -slab@0.4.9 X -smallvec@1.11.2 X X -socket2@0.5.5 X X -spin@0.5.2 X -spin@0.9.8 X -spki@0.7.3 X X -strsim@0.10.0 X -strum@0.25.0 X -strum_macros@0.25.3 X -subtle@2.5.0 X -syn@1.0.109 X X -syn@2.0.43 X X -system-configuration@0.5.1 X X -system-configuration-sys@0.5.0 X X -tap@1.0.1 X -tempfile@3.8.1 X X -thiserror@1.0.52 X X -thiserror-impl@1.0.52 X X -time@0.3.31 X X -time-core@0.1.2 X X -time-macros@0.2.16 X X -tiny-keccak@2.0.2 X -tinyvec@1.6.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.35.1 X -tokio-macros@2.2.0 X -tokio-native-tls@0.3.1 X -tokio-rustls@0.24.1 X X -tokio-util@0.7.10 X -tower-service@0.3.2 X -tracing@0.1.40 X -tracing-core@0.1.32 X -try-lock@0.2.5 X -typed-builder@0.16.2 X X -typed-builder@0.18.0 X X -typed-builder-macro@0.16.2 X X -typed-builder-macro@0.18.0 X X -typenum@1.17.0 X X -unicode-bidi@0.3.14 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.22 X X -untrusted@0.9.0 X -url@2.5.0 X X -urlencoding@2.1.3 X -utf8parse@0.2.1 X X -uuid@1.6.1 X X -vcpkg@0.2.15 X X -version_check@0.9.4 X X -want@0.3.1 X -wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.89 X X -wasm-bindgen-backend@0.2.89 X X -wasm-bindgen-futures@0.4.39 X X -wasm-bindgen-macro@0.2.89 X X -wasm-bindgen-macro-support@0.2.89 X X -wasm-bindgen-shared@0.2.89 X X -wasm-streams@0.3.0 X X -web-sys@0.3.66 X X -windows-core@0.51.1 X X -windows-sys@0.48.0 X X -windows-sys@0.52.0 X X -windows-targets@0.48.5 X X -windows-targets@0.52.0 X X -windows_aarch64_gnullvm@0.48.5 X X -windows_aarch64_gnullvm@0.52.0 X X -windows_aarch64_msvc@0.48.5 X X -windows_aarch64_msvc@0.52.0 X X -windows_i686_gnu@0.48.5 X X -windows_i686_gnu@0.52.0 X X -windows_i686_msvc@0.48.5 X X -windows_i686_msvc@0.52.0 X X -windows_x86_64_gnu@0.48.5 X X -windows_x86_64_gnu@0.52.0 X X -windows_x86_64_gnullvm@0.48.5 X X -windows_x86_64_gnullvm@0.52.0 X X -windows_x86_64_msvc@0.48.5 X X -windows_x86_64_msvc@0.52.0 X X -winreg@0.50.0 X -wyz@0.5.1 X -zerocopy@0.7.32 X X X -zeroize@1.7.0 X X diff --git a/crates/rest/src/catalog.rs b/crates/rest/src/catalog.rs deleted file mode 100644 index c10d904..0000000 --- a/crates/rest/src/catalog.rs +++ /dev/null @@ -1,1604 +0,0 @@ -// 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. - -//! This module contains rest catalog implementation. - -use std::collections::HashMap; - -use async_trait::async_trait; -use reqwest::header::{self, HeaderMap, HeaderName, HeaderValue}; -use reqwest::{Client, Request, Response, StatusCode}; -use serde::de::DeserializeOwned; -use typed_builder::TypedBuilder; -use urlencoding::encode; - -use crate::catalog::_serde::{ - CommitTableRequest, CommitTableResponse, CreateTableRequest, LoadTableResponse, -}; -use iceberg::io::FileIO; -use iceberg::table::Table; -use iceberg::Result; -use iceberg::{ - Catalog, Error, ErrorKind, Namespace, NamespaceIdent, TableCommit, TableCreation, TableIdent, -}; - -use self::_serde::{ - CatalogConfig, ErrorResponse, ListNamespaceResponse, ListTableResponse, NamespaceSerde, - RenameTableRequest, NO_CONTENT, OK, -}; - -const ICEBERG_REST_SPEC_VERSION: &str = "0.14.1"; -const CARGO_PKG_VERSION: &str = env!("CARGO_PKG_VERSION"); -const PATH_V1: &str = "v1"; - -/// Rest catalog configuration. -#[derive(Debug, TypedBuilder)] -pub struct RestCatalogConfig { - uri: String, - #[builder(default, setter(strip_option))] - warehouse: Option, - - #[builder(default)] - props: HashMap, -} - -impl RestCatalogConfig { - fn config_endpoint(&self) -> String { - [&self.uri, PATH_V1, "config"].join("/") - } - - fn namespaces_endpoint(&self) -> String { - [&self.uri, PATH_V1, "namespaces"].join("/") - } - - fn namespace_endpoint(&self, ns: &NamespaceIdent) -> String { - [&self.uri, PATH_V1, "namespaces", &ns.encode_in_url()].join("/") - } - - fn tables_endpoint(&self, ns: &NamespaceIdent) -> String { - [ - &self.uri, - PATH_V1, - "namespaces", - &ns.encode_in_url(), - "tables", - ] - .join("/") - } - - fn rename_table_endpoint(&self) -> String { - [&self.uri, PATH_V1, "tables", "rename"].join("/") - } - - fn table_endpoint(&self, table: &TableIdent) -> String { - [ - &self.uri, - PATH_V1, - "namespaces", - &table.namespace.encode_in_url(), - "tables", - encode(&table.name).as_ref(), - ] - .join("/") - } - - fn try_create_rest_client(&self) -> Result { - //TODO: We will add oauth, ssl config, sigv4 later - let headers = HeaderMap::from_iter([ - ( - header::CONTENT_TYPE, - HeaderValue::from_static("application/json"), - ), - ( - HeaderName::from_static("x-client-version"), - HeaderValue::from_static(ICEBERG_REST_SPEC_VERSION), - ), - ( - header::USER_AGENT, - HeaderValue::from_str(&format!("iceberg-rs/{}", CARGO_PKG_VERSION)).unwrap(), - ), - ]); - - Ok(HttpClient( - Client::builder().default_headers(headers).build()?, - )) - } -} - -#[derive(Debug)] -struct HttpClient(Client); - -impl HttpClient { - async fn query< - R: DeserializeOwned, - E: DeserializeOwned + Into, - const SUCCESS_CODE: u16, - >( - &self, - request: Request, - ) -> Result { - let resp = self.0.execute(request).await?; - - if resp.status().as_u16() == SUCCESS_CODE { - let text = resp.bytes().await?; - Ok(serde_json::from_slice::(&text).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to parse response from rest catalog server!", - ) - .with_context("json", String::from_utf8_lossy(&text)) - .with_source(e) - })?) - } else { - let text = resp.bytes().await?; - let e = serde_json::from_slice::(&text).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to parse response from rest catalog server!", - ) - .with_context("json", String::from_utf8_lossy(&text)) - .with_source(e) - })?; - Err(e.into()) - } - } - - async fn execute, const SUCCESS_CODE: u16>( - &self, - request: Request, - ) -> Result<()> { - let resp = self.0.execute(request).await?; - - if resp.status().as_u16() == SUCCESS_CODE { - Ok(()) - } else { - let code = resp.status(); - let text = resp.bytes().await?; - let e = serde_json::from_slice::(&text).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to parse response from rest catalog server!", - ) - .with_context("json", String::from_utf8_lossy(&text)) - .with_context("code", code.to_string()) - .with_source(e) - })?; - Err(e.into()) - } - } - - /// More generic logic handling for special cases like head. - async fn do_execute>( - &self, - request: Request, - handler: impl FnOnce(&Response) -> Option, - ) -> Result { - let resp = self.0.execute(request).await?; - - if let Some(ret) = handler(&resp) { - Ok(ret) - } else { - let code = resp.status(); - let text = resp.bytes().await?; - let e = serde_json::from_slice::(&text).map_err(|e| { - Error::new( - ErrorKind::Unexpected, - "Failed to parse response from rest catalog server!", - ) - .with_context("code", code.to_string()) - .with_context("json", String::from_utf8_lossy(&text)) - .with_source(e) - })?; - Err(e.into()) - } - } -} - -/// Rest catalog implementation. -#[derive(Debug)] -pub struct RestCatalog { - config: RestCatalogConfig, - client: HttpClient, -} - -#[async_trait] -impl Catalog for RestCatalog { - /// List namespaces from table. - async fn list_namespaces( - &self, - parent: Option<&NamespaceIdent>, - ) -> Result> { - let mut request = self.client.0.get(self.config.namespaces_endpoint()); - if let Some(ns) = parent { - request = request.query(&[("parent", ns.encode_in_url())]); - } - - let resp = self - .client - .query::(request.build()?) - .await?; - - resp.namespaces - .into_iter() - .map(NamespaceIdent::from_vec) - .collect::>>() - } - - /// Create a new namespace inside the catalog. - async fn create_namespace( - &self, - namespace: &NamespaceIdent, - properties: HashMap, - ) -> Result { - let request = self - .client - .0 - .post(self.config.namespaces_endpoint()) - .json(&NamespaceSerde { - namespace: namespace.as_ref().clone(), - properties: Some(properties), - }) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - - Namespace::try_from(resp) - } - - /// Get a namespace information from the catalog. - async fn get_namespace(&self, namespace: &NamespaceIdent) -> Result { - let request = self - .client - .0 - .get(self.config.namespace_endpoint(namespace)) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - Namespace::try_from(resp) - } - - /// Update a namespace inside the catalog. - /// - /// # Behavior - /// - /// The properties must be the full set of namespace. - async fn update_namespace( - &self, - _namespace: &NamespaceIdent, - _properties: HashMap, - ) -> Result<()> { - Err(Error::new( - ErrorKind::FeatureUnsupported, - "Updating namespace not supported yet!", - )) - } - - async fn namespace_exists(&self, ns: &NamespaceIdent) -> Result { - let request = self - .client - .0 - .head(self.config.namespace_endpoint(ns)) - .build()?; - - self.client - .do_execute::(request, |resp| match resp.status() { - StatusCode::NO_CONTENT => Some(true), - StatusCode::NOT_FOUND => Some(false), - _ => None, - }) - .await - } - - /// Drop a namespace from the catalog. - async fn drop_namespace(&self, namespace: &NamespaceIdent) -> Result<()> { - let request = self - .client - .0 - .delete(self.config.namespace_endpoint(namespace)) - .build()?; - - self.client - .execute::(request) - .await - } - - /// List tables from namespace. - async fn list_tables(&self, namespace: &NamespaceIdent) -> Result> { - let request = self - .client - .0 - .get(self.config.tables_endpoint(namespace)) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - - Ok(resp.identifiers) - } - - /// Create a new table inside the namespace. - async fn create_table( - &self, - namespace: &NamespaceIdent, - creation: TableCreation, - ) -> Result
{ - let table_ident = TableIdent::new(namespace.clone(), creation.name.clone()); - - let request = self - .client - .0 - .post(self.config.tables_endpoint(namespace)) - .json(&CreateTableRequest { - name: creation.name, - location: creation.location, - schema: creation.schema, - partition_spec: creation.partition_spec, - write_order: creation.sort_order, - // We don't support stage create yet. - stage_create: Some(false), - properties: if creation.properties.is_empty() { - None - } else { - Some(creation.properties) - }, - }) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - - let file_io = self.load_file_io(resp.metadata_location.as_deref(), resp.config)?; - - let table = Table::builder() - .identifier(table_ident) - .file_io(file_io) - .metadata(resp.metadata) - .metadata_location(resp.metadata_location.ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - "Metadata location missing in create table response!", - ) - })?) - .build(); - - Ok(table) - } - - /// Load table from the catalog. - async fn load_table(&self, table: &TableIdent) -> Result
{ - let request = self - .client - .0 - .get(self.config.table_endpoint(table)) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - - let file_io = self.load_file_io(resp.metadata_location.as_deref(), resp.config)?; - - let table_builder = Table::builder() - .identifier(table.clone()) - .file_io(file_io) - .metadata(resp.metadata); - - if let Some(metadata_location) = resp.metadata_location { - Ok(table_builder.metadata_location(metadata_location).build()) - } else { - Ok(table_builder.build()) - } - } - - /// Drop a table from the catalog. - async fn drop_table(&self, table: &TableIdent) -> Result<()> { - let request = self - .client - .0 - .delete(self.config.table_endpoint(table)) - .build()?; - - self.client - .execute::(request) - .await - } - - /// Check if a table exists in the catalog. - async fn stat_table(&self, table: &TableIdent) -> Result { - let request = self - .client - .0 - .head(self.config.table_endpoint(table)) - .build()?; - - self.client - .do_execute::(request, |resp| match resp.status() { - StatusCode::NO_CONTENT => Some(true), - StatusCode::NOT_FOUND => Some(false), - _ => None, - }) - .await - } - - /// Rename a table in the catalog. - async fn rename_table(&self, src: &TableIdent, dest: &TableIdent) -> Result<()> { - let request = self - .client - .0 - .post(self.config.rename_table_endpoint()) - .json(&RenameTableRequest { - source: src.clone(), - destination: dest.clone(), - }) - .build()?; - - self.client - .execute::(request) - .await - } - - /// Update table. - async fn update_table(&self, mut commit: TableCommit) -> Result
{ - let request = self - .client - .0 - .post(self.config.table_endpoint(commit.identifier())) - .json(&CommitTableRequest { - identifier: commit.identifier().clone(), - requirements: commit.take_requirements(), - updates: commit.take_updates(), - }) - .build()?; - - let resp = self - .client - .query::(request) - .await?; - - let file_io = self.load_file_io(Some(&resp.metadata_location), None)?; - Ok(Table::builder() - .identifier(commit.identifier().clone()) - .file_io(file_io) - .metadata(resp.metadata) - .metadata_location(resp.metadata_location) - .build()) - } -} - -impl RestCatalog { - /// Creates a rest catalog from config. - pub async fn new(config: RestCatalogConfig) -> Result { - let mut catalog = Self { - client: config.try_create_rest_client()?, - config, - }; - - catalog.update_config().await?; - catalog.client = catalog.config.try_create_rest_client()?; - - Ok(catalog) - } - - async fn update_config(&mut self) -> Result<()> { - let mut request = self.client.0.get(self.config.config_endpoint()); - - if let Some(warehouse_location) = &self.config.warehouse { - request = request.query(&[("warehouse", warehouse_location)]); - } - - let config = self - .client - .query::(request.build()?) - .await?; - - let mut props = config.defaults; - props.extend(self.config.props.clone()); - props.extend(config.overrides); - - self.config.props = props; - - Ok(()) - } - - fn load_file_io( - &self, - metadata_location: Option<&str>, - extra_config: Option>, - ) -> Result { - let mut props = self.config.props.clone(); - if let Some(config) = extra_config { - props.extend(config); - } - - let file_io = match self.config.warehouse.as_deref().or(metadata_location) { - Some(url) => FileIO::from_path(url)?.with_props(props).build()?, - None => { - return Err(Error::new( - ErrorKind::Unexpected, - "Unable to load file io, neither warehouse nor metadata location is set!", - ))? - } - }; - - Ok(file_io) - } -} - -/// Requests and responses for rest api. -mod _serde { - use std::collections::HashMap; - - use serde_derive::{Deserialize, Serialize}; - - use iceberg::spec::{Schema, SortOrder, TableMetadata, UnboundPartitionSpec}; - use iceberg::{Error, ErrorKind, Namespace, TableIdent, TableRequirement, TableUpdate}; - - pub(super) const OK: u16 = 200u16; - pub(super) const NO_CONTENT: u16 = 204u16; - - #[derive(Clone, Debug, Serialize, Deserialize)] - pub(super) struct CatalogConfig { - pub(super) overrides: HashMap, - pub(super) defaults: HashMap, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct ErrorResponse { - error: ErrorModel, - } - - impl From for Error { - fn from(resp: ErrorResponse) -> Error { - resp.error.into() - } - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct ErrorModel { - pub(super) message: String, - pub(super) r#type: String, - pub(super) code: u16, - pub(super) stack: Option>, - } - - impl From for Error { - fn from(value: ErrorModel) -> Self { - let mut error = Error::new(ErrorKind::DataInvalid, value.message) - .with_context("type", value.r#type) - .with_context("code", format!("{}", value.code)); - - if let Some(stack) = value.stack { - error = error.with_context("stack", stack.join("\n")); - } - - error - } - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct OAuthError { - pub(super) error: String, - pub(super) error_description: Option, - pub(super) error_uri: Option, - } - - impl From for Error { - fn from(value: OAuthError) -> Self { - let mut error = Error::new( - ErrorKind::DataInvalid, - format!("OAuthError: {}", value.error), - ); - - if let Some(desc) = value.error_description { - error = error.with_context("description", desc); - } - - if let Some(uri) = value.error_uri { - error = error.with_context("uri", uri); - } - - error - } - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct NamespaceSerde { - pub(super) namespace: Vec, - pub(super) properties: Option>, - } - - impl TryFrom for super::Namespace { - type Error = Error; - fn try_from(value: NamespaceSerde) -> std::result::Result { - Ok(super::Namespace::with_properties( - super::NamespaceIdent::from_vec(value.namespace)?, - value.properties.unwrap_or_default(), - )) - } - } - - impl From<&Namespace> for NamespaceSerde { - fn from(value: &Namespace) -> Self { - Self { - namespace: value.name().as_ref().clone(), - properties: Some(value.properties().clone()), - } - } - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct ListNamespaceResponse { - pub(super) namespaces: Vec>, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct UpdateNamespacePropsRequest { - removals: Option>, - updates: Option>, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct UpdateNamespacePropsResponse { - updated: Vec, - removed: Vec, - missing: Option>, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct ListTableResponse { - pub(super) identifiers: Vec, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct RenameTableRequest { - pub(super) source: TableIdent, - pub(super) destination: TableIdent, - } - - #[derive(Debug, Deserialize)] - #[serde(rename_all = "kebab-case")] - pub(super) struct LoadTableResponse { - pub(super) metadata_location: Option, - pub(super) metadata: TableMetadata, - pub(super) config: Option>, - } - - #[derive(Debug, Serialize, Deserialize)] - #[serde(rename_all = "kebab-case")] - pub(super) struct CreateTableRequest { - pub(super) name: String, - pub(super) location: Option, - pub(super) schema: Schema, - pub(super) partition_spec: Option, - pub(super) write_order: Option, - pub(super) stage_create: Option, - pub(super) properties: Option>, - } - - #[derive(Debug, Serialize, Deserialize)] - pub(super) struct CommitTableRequest { - pub(super) identifier: TableIdent, - pub(super) requirements: Vec, - pub(super) updates: Vec, - } - - #[derive(Debug, Serialize, Deserialize)] - #[serde(rename_all = "kebab-case")] - pub(super) struct CommitTableResponse { - pub(super) metadata_location: String, - pub(super) metadata: TableMetadata, - } -} - -#[cfg(test)] -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, - }; - use iceberg::transaction::Transaction; - use mockito::{Mock, Server, ServerGuard}; - use std::fs::File; - use std::io::BufReader; - use std::sync::Arc; - use uuid::uuid; - - use super::*; - - #[tokio::test] - async fn test_update_config() { - let mut server = Server::new_async().await; - - let config_mock = server - .mock("GET", "/v1/config") - .with_status(200) - .with_body( - r#"{ - "overrides": { - "warehouse": "s3://iceberg-catalog" - }, - "defaults": {} - }"#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - assert_eq!( - catalog.config.props.get("warehouse"), - Some(&"s3://iceberg-catalog".to_string()) - ); - - config_mock.assert_async().await; - } - - async fn create_config_mock(server: &mut ServerGuard) -> Mock { - server - .mock("GET", "/v1/config") - .with_status(200) - .with_body( - r#"{ - "overrides": { - "warehouse": "s3://iceberg-catalog" - }, - "defaults": {} - }"#, - ) - .create_async() - .await - } - - #[tokio::test] - async fn test_list_namespace() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let list_ns_mock = server - .mock("GET", "/v1/namespaces") - .with_body( - r#"{ - "namespaces": [ - ["ns1", "ns11"], - ["ns2"] - ] - }"#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let namespaces = catalog.list_namespaces(None).await.unwrap(); - - let expected_ns = vec![ - NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), - NamespaceIdent::from_vec(vec!["ns2".to_string()]).unwrap(), - ]; - - assert_eq!(expected_ns, namespaces); - - config_mock.assert_async().await; - list_ns_mock.assert_async().await; - } - - #[tokio::test] - async fn test_create_namespace() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let create_ns_mock = server - .mock("POST", "/v1/namespaces") - .with_body( - r#"{ - "namespace": [ "ns1", "ns11"], - "properties" : { - "key1": "value1" - } - }"#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let namespaces = catalog - .create_namespace( - &NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), - HashMap::from([("key1".to_string(), "value1".to_string())]), - ) - .await - .unwrap(); - - let expected_ns = Namespace::with_properties( - NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(), - HashMap::from([("key1".to_string(), "value1".to_string())]), - ); - - assert_eq!(expected_ns, namespaces); - - config_mock.assert_async().await; - create_ns_mock.assert_async().await; - } - - #[tokio::test] - async fn test_get_namespace() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let get_ns_mock = server - .mock("GET", "/v1/namespaces/ns1") - .with_body( - r#"{ - "namespace": [ "ns1"], - "properties" : { - "key1": "value1" - } - }"#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let namespaces = catalog - .get_namespace(&NamespaceIdent::new("ns1".to_string())) - .await - .unwrap(); - - let expected_ns = Namespace::with_properties( - NamespaceIdent::new("ns1".to_string()), - HashMap::from([("key1".to_string(), "value1".to_string())]), - ); - - assert_eq!(expected_ns, namespaces); - - config_mock.assert_async().await; - get_ns_mock.assert_async().await; - } - - #[tokio::test] - async fn check_namespace_exists() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let get_ns_mock = server - .mock("HEAD", "/v1/namespaces/ns1") - .with_status(204) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - assert!(catalog - .namespace_exists(&NamespaceIdent::new("ns1".to_string())) - .await - .unwrap()); - - config_mock.assert_async().await; - get_ns_mock.assert_async().await; - } - - #[tokio::test] - async fn test_drop_namespace() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let drop_ns_mock = server - .mock("DELETE", "/v1/namespaces/ns1") - .with_status(204) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - catalog - .drop_namespace(&NamespaceIdent::new("ns1".to_string())) - .await - .unwrap(); - - config_mock.assert_async().await; - drop_ns_mock.assert_async().await; - } - - #[tokio::test] - async fn test_list_tables() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let list_tables_mock = server - .mock("GET", "/v1/namespaces/ns1/tables") - .with_status(200) - .with_body( - r#"{ - "identifiers": [ - { - "namespace": ["ns1"], - "name": "table1" - }, - { - "namespace": ["ns1"], - "name": "table2" - } - ] - }"#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let tables = catalog - .list_tables(&NamespaceIdent::new("ns1".to_string())) - .await - .unwrap(); - - let expected_tables = vec![ - TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table1".to_string()), - TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table2".to_string()), - ]; - - assert_eq!(tables, expected_tables); - - config_mock.assert_async().await; - list_tables_mock.assert_async().await; - } - - #[tokio::test] - async fn test_drop_tables() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let delete_table_mock = server - .mock("DELETE", "/v1/namespaces/ns1/tables/table1") - .with_status(204) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - catalog - .drop_table(&TableIdent::new( - NamespaceIdent::new("ns1".to_string()), - "table1".to_string(), - )) - .await - .unwrap(); - - config_mock.assert_async().await; - delete_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_check_table_exists() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let check_table_exists_mock = server - .mock("HEAD", "/v1/namespaces/ns1/tables/table1") - .with_status(204) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - assert!(catalog - .stat_table(&TableIdent::new( - NamespaceIdent::new("ns1".to_string()), - "table1".to_string(), - )) - .await - .unwrap()); - - config_mock.assert_async().await; - check_table_exists_mock.assert_async().await; - } - - #[tokio::test] - async fn test_rename_table() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let rename_table_mock = server - .mock("POST", "/v1/tables/rename") - .with_status(204) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - catalog - .rename_table( - &TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table1".to_string()), - &TableIdent::new(NamespaceIdent::new("ns1".to_string()), "table2".to_string()), - ) - .await - .unwrap(); - - config_mock.assert_async().await; - rename_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_load_table() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let rename_table_mock = server - .mock("GET", "/v1/namespaces/ns1/tables/test1") - .with_status(200) - .with_body_from_file(format!( - "{}/testdata/{}", - env!("CARGO_MANIFEST_DIR"), - "load_table_response.json" - )) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let table = catalog - .load_table(&TableIdent::new( - NamespaceIdent::new("ns1".to_string()), - "test1".to_string(), - )) - .await - .unwrap(); - - assert_eq!( - &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), - table.identifier() - ); - assert_eq!("s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json", table.metadata_location().unwrap()); - assert_eq!(FormatVersion::V1, table.metadata().format_version()); - assert_eq!("s3://warehouse/database/table", table.metadata().location()); - assert_eq!( - uuid!("b55d9dda-6561-423a-8bfc-787980ce421f"), - table.metadata().uuid() - ); - assert_eq!( - Utc.timestamp_millis_opt(1646787054459).unwrap(), - table.metadata().last_updated_ms() - ); - assert_eq!( - vec![&Arc::new( - Schema::builder() - .with_fields(vec![ - NestedField::optional(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "data", Type::Primitive(PrimitiveType::String)) - .into(), - ]) - .build() - .unwrap() - )], - table.metadata().schemas_iter().collect::>() - ); - assert_eq!( - &HashMap::from([ - ("owner".to_string(), "bryan".to_string()), - ( - "write.metadata.compression-codec".to_string(), - "gzip".to_string() - ) - ]), - table.metadata().properties() - ); - assert_eq!(vec![&Arc::new(Snapshot::builder() - .with_snapshot_id(3497810964824022504) - .with_timestamp_ms(1646787054459) - .with_manifest_list("s3://warehouse/database/table/metadata/snap-3497810964824022504-1-c4f68204-666b-4e50-a9df-b10c34bf6b82.avro") - .with_sequence_number(0) - .with_schema_id(0) - .with_summary(Summary { - operation: Operation::Append, - other: HashMap::from_iter([ - ("spark.app.id", "local-1646787004168"), - ("added-data-files", "1"), - ("added-records", "1"), - ("added-files-size", "697"), - ("changed-partition-count", "1"), - ("total-records", "1"), - ("total-files-size", "697"), - ("total-data-files", "1"), - ("total-delete-files", "0"), - ("total-position-deletes", "0"), - ("total-equality-deletes", "0") - ].iter().map(|p| (p.0.to_string(), p.1.to_string()))), - }).build() - )], table.metadata().snapshots().collect::>()); - assert_eq!( - &[SnapshotLog { - timestamp_ms: 1646787054459, - snapshot_id: 3497810964824022504, - }], - table.metadata().history() - ); - assert_eq!( - vec![&Arc::new(SortOrder { - order_id: 0, - fields: vec![], - })], - table.metadata().sort_orders_iter().collect::>() - ); - - config_mock.assert_async().await; - rename_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_load_table_404() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let rename_table_mock = server - .mock("GET", "/v1/namespaces/ns1/tables/test1") - .with_status(404) - .with_body(r#" -{ - "error": { - "message": "Table does not exist: ns1.test1 in warehouse 8bcb0838-50fc-472d-9ddb-8feb89ef5f1e", - "type": "NoSuchNamespaceErrorException", - "code": 404 - } -} - "#) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let table = catalog - .load_table(&TableIdent::new( - NamespaceIdent::new("ns1".to_string()), - "test1".to_string(), - )) - .await; - - assert!(table.is_err()); - assert!(table - .err() - .unwrap() - .message() - .contains("Table does not exist")); - - config_mock.assert_async().await; - rename_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_create_table() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let create_table_mock = server - .mock("POST", "/v1/namespaces/ns1/tables") - .with_status(200) - .with_body_from_file(format!( - "{}/testdata/{}", - env!("CARGO_MANIFEST_DIR"), - "create_table_response.json" - )) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .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(), - ) - .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) - .partition_spec( - UnboundPartitionSpec::builder() - .with_fields(vec![UnboundPartitionField::builder() - .source_id(1) - .transform(Transform::Truncate(3)) - .name("id".to_string()) - .build()]) - .build() - .unwrap(), - ) - .sort_order( - SortOrder::builder() - .with_sort_field( - SortField::builder() - .source_id(2) - .transform(Transform::Identity) - .direction(SortDirection::Ascending) - .null_order(NullOrder::First) - .build(), - ) - .build_unbound() - .unwrap(), - ) - .build(); - - let table = catalog - .create_table(&NamespaceIdent::from_strs(["ns1"]).unwrap(), table_creation) - .await - .unwrap(); - - assert_eq!( - &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), - table.identifier() - ); - assert_eq!( - "s3://warehouse/database/table/metadata.json", - table.metadata_location().unwrap() - ); - assert_eq!(FormatVersion::V1, table.metadata().format_version()); - assert_eq!("s3://warehouse/database/table", table.metadata().location()); - assert_eq!( - uuid!("bf289591-dcc0-4234-ad4f-5c3eed811a29"), - table.metadata().uuid() - ); - assert_eq!( - 1657810967051, - table.metadata().last_updated_ms().timestamp_millis() - ); - assert_eq!( - vec![&Arc::new( - 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(0) - .with_identifier_field_ids(vec![2]) - .build() - .unwrap() - )], - table.metadata().schemas_iter().collect::>() - ); - assert_eq!( - &HashMap::from([ - ( - "write.delete.parquet.compression-codec".to_string(), - "zstd".to_string() - ), - ( - "write.metadata.compression-codec".to_string(), - "gzip".to_string() - ), - ( - "write.summary.partition-limit".to_string(), - "100".to_string() - ), - ( - "write.parquet.compression-codec".to_string(), - "zstd".to_string() - ), - ]), - table.metadata().properties() - ); - assert!(table.metadata().current_snapshot().is_none()); - assert!(table.metadata().history().is_empty()); - assert_eq!( - vec![&Arc::new(SortOrder { - order_id: 0, - fields: vec![], - })], - table.metadata().sort_orders_iter().collect::>() - ); - - config_mock.assert_async().await; - create_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_create_table_409() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let create_table_mock = server - .mock("POST", "/v1/namespaces/ns1/tables") - .with_status(409) - .with_body(r#" -{ - "error": { - "message": "Table already exists: ns1.test1 in warehouse 8bcb0838-50fc-472d-9ddb-8feb89ef5f1e", - "type": "AlreadyExistsException", - "code": 409 - } -} - "#) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .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(), - ) - .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) - .build(); - - let table_result = catalog - .create_table(&NamespaceIdent::from_strs(["ns1"]).unwrap(), table_creation) - .await; - - assert!(table_result.is_err()); - assert!(table_result - .err() - .unwrap() - .message() - .contains("Table already exists")); - - config_mock.assert_async().await; - create_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_update_table() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let update_table_mock = server - .mock("POST", "/v1/namespaces/ns1/tables/test1") - .with_status(200) - .with_body_from_file(format!( - "{}/testdata/{}", - env!("CARGO_MANIFEST_DIR"), - "update_table_response.json" - )) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let table1 = { - let file = File::open(format!( - "{}/testdata/{}", - env!("CARGO_MANIFEST_DIR"), - "create_table_response.json" - )) - .unwrap(); - let reader = BufReader::new(file); - let resp = serde_json::from_reader::<_, LoadTableResponse>(reader).unwrap(); - - Table::builder() - .metadata(resp.metadata) - .metadata_location(resp.metadata_location.unwrap()) - .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) - .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) - .build() - }; - - let table = Transaction::new(&table1) - .upgrade_table_version(FormatVersion::V2) - .unwrap() - .commit(&catalog) - .await - .unwrap(); - - assert_eq!( - &TableIdent::from_strs(vec!["ns1", "test1"]).unwrap(), - table.identifier() - ); - assert_eq!( - "s3://warehouse/database/table/metadata.json", - table.metadata_location().unwrap() - ); - assert_eq!(FormatVersion::V2, table.metadata().format_version()); - assert_eq!("s3://warehouse/database/table", table.metadata().location()); - assert_eq!( - uuid!("bf289591-dcc0-4234-ad4f-5c3eed811a29"), - table.metadata().uuid() - ); - assert_eq!( - 1657810967051, - table.metadata().last_updated_ms().timestamp_millis() - ); - assert_eq!( - vec![&Arc::new( - 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(0) - .with_identifier_field_ids(vec![2]) - .build() - .unwrap() - )], - table.metadata().schemas_iter().collect::>() - ); - assert_eq!( - &HashMap::from([ - ( - "write.delete.parquet.compression-codec".to_string(), - "zstd".to_string() - ), - ( - "write.metadata.compression-codec".to_string(), - "gzip".to_string() - ), - ( - "write.summary.partition-limit".to_string(), - "100".to_string() - ), - ( - "write.parquet.compression-codec".to_string(), - "zstd".to_string() - ), - ]), - table.metadata().properties() - ); - assert!(table.metadata().current_snapshot().is_none()); - assert!(table.metadata().history().is_empty()); - assert_eq!( - vec![&Arc::new(SortOrder { - order_id: 0, - fields: vec![], - })], - table.metadata().sort_orders_iter().collect::>() - ); - - config_mock.assert_async().await; - update_table_mock.assert_async().await; - } - - #[tokio::test] - async fn test_update_table_404() { - let mut server = Server::new_async().await; - - let config_mock = create_config_mock(&mut server).await; - - let update_table_mock = server - .mock("POST", "/v1/namespaces/ns1/tables/test1") - .with_status(404) - .with_body( - r#" -{ - "error": { - "message": "The given table does not exist", - "type": "NoSuchTableException", - "code": 404 - } -} - "#, - ) - .create_async() - .await; - - let catalog = RestCatalog::new(RestCatalogConfig::builder().uri(server.url()).build()) - .await - .unwrap(); - - let table1 = { - let file = File::open(format!( - "{}/testdata/{}", - env!("CARGO_MANIFEST_DIR"), - "create_table_response.json" - )) - .unwrap(); - let reader = BufReader::new(file); - let resp = serde_json::from_reader::<_, LoadTableResponse>(reader).unwrap(); - - Table::builder() - .metadata(resp.metadata) - .metadata_location(resp.metadata_location.unwrap()) - .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) - .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) - .build() - }; - - let table_result = Transaction::new(&table1) - .upgrade_table_version(FormatVersion::V2) - .unwrap() - .commit(&catalog) - .await; - - assert!(table_result.is_err()); - assert!(table_result - .err() - .unwrap() - .message() - .contains("The given table does not exist")); - - config_mock.assert_async().await; - update_table_mock.assert_async().await; - } -} diff --git a/crates/rest/src/lib.rs b/crates/rest/src/lib.rs deleted file mode 100644 index 023fe7a..0000000 --- a/crates/rest/src/lib.rs +++ /dev/null @@ -1,23 +0,0 @@ -// 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. - -//! Iceberg REST API implementation. - -#![deny(missing_docs)] - -mod catalog; -pub use catalog::*; diff --git a/crates/rest/testdata/create_table_response.json b/crates/rest/testdata/create_table_response.json deleted file mode 100644 index e01a52f..0000000 --- a/crates/rest/testdata/create_table_response.json +++ /dev/null @@ -1,53 +0,0 @@ -{ - "metadata-location": "s3://warehouse/database/table/metadata.json", - "metadata": { - "format-version": 1, - "table-uuid": "bf289591-dcc0-4234-ad4f-5c3eed811a29", - "location": "s3://warehouse/database/table", - "last-updated-ms": 1657810967051, - "last-column-id": 3, - "schema": { - "type": "struct", - "schema-id": 0, - "identifier-field-ids": [2], - "fields": [ - {"id": 1, "name": "foo", "required": false, "type": "string"}, - {"id": 2, "name": "bar", "required": true, "type": "int"}, - {"id": 3, "name": "baz", "required": false, "type": "boolean"} - ] - }, - "current-schema-id": 0, - "schemas": [ - { - "type": "struct", - "schema-id": 0, - "identifier-field-ids": [2], - "fields": [ - {"id": 1, "name": "foo", "required": false, "type": "string"}, - {"id": 2, "name": "bar", "required": true, "type": "int"}, - {"id": 3, "name": "baz", "required": false, "type": "boolean"} - ] - } - ], - "partition-spec": [], - "default-spec-id": 0, - "last-partition-id": 999, - "default-sort-order-id": 0, - "sort-orders": [{"order-id": 0, "fields": []}], - "properties": { - "write.delete.parquet.compression-codec": "zstd", - "write.metadata.compression-codec": "gzip", - "write.summary.partition-limit": "100", - "write.parquet.compression-codec": "zstd" - }, - "current-snapshot-id": -1, - "refs": {}, - "snapshots": [], - "snapshot-log": [], - "metadata-log": [] - }, - "config": { - "client.factory": "io.tabular.iceberg.catalog.TabularAwsClientFactory", - "region": "us-west-2" - } -} \ No newline at end of file diff --git a/crates/rest/testdata/load_table_response.json b/crates/rest/testdata/load_table_response.json deleted file mode 100644 index 012f0e9..0000000 --- a/crates/rest/testdata/load_table_response.json +++ /dev/null @@ -1,68 +0,0 @@ -{ - "metadata-location": "s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json", - "metadata": { - "format-version": 1, - "table-uuid": "b55d9dda-6561-423a-8bfc-787980ce421f", - "location": "s3://warehouse/database/table", - "last-updated-ms": 1646787054459, - "last-column-id": 2, - "schema": { - "type": "struct", - "schema-id": 0, - "fields": [ - {"id": 1, "name": "id", "required": false, "type": "int"}, - {"id": 2, "name": "data", "required": false, "type": "string"} - ] - }, - "current-schema-id": 0, - "schemas": [ - { - "type": "struct", - "schema-id": 0, - "fields": [ - {"id": 1, "name": "id", "required": false, "type": "int"}, - {"id": 2, "name": "data", "required": false, "type": "string"} - ] - } - ], - "partition-spec": [], - "default-spec-id": 0, - "partition-specs": [{"spec-id": 0, "fields": []}], - "last-partition-id": 999, - "default-sort-order-id": 0, - "sort-orders": [{"order-id": 0, "fields": []}], - "properties": {"owner": "bryan", "write.metadata.compression-codec": "gzip"}, - "current-snapshot-id": 3497810964824022504, - "refs": {"main": {"snapshot-id": 3497810964824022504, "type": "branch"}}, - "snapshots": [ - { - "snapshot-id": 3497810964824022504, - "timestamp-ms": 1646787054459, - "summary": { - "operation": "append", - "spark.app.id": "local-1646787004168", - "added-data-files": "1", - "added-records": "1", - "added-files-size": "697", - "changed-partition-count": "1", - "total-records": "1", - "total-files-size": "697", - "total-data-files": "1", - "total-delete-files": "0", - "total-position-deletes": "0", - "total-equality-deletes": "0" - }, - "manifest-list": "s3://warehouse/database/table/metadata/snap-3497810964824022504-1-c4f68204-666b-4e50-a9df-b10c34bf6b82.avro", - "schema-id": 0 - } - ], - "snapshot-log": [{"timestamp-ms": 1646787054459, "snapshot-id": 3497810964824022504}], - "metadata-log": [ - { - "timestamp-ms": 1646787031514, - "metadata-file": "s3://warehouse/database/table/metadata/00000-88484a1c-00e5-4a07-a787-c0e7aeffa805.gz.metadata.json" - } - ] - }, - "config": {"client.factory": "io.tabular.iceberg.catalog.TabularAwsClientFactory", "region": "us-west-2"} -} \ No newline at end of file diff --git a/crates/rest/testdata/rest_catalog/docker-compose.yaml b/crates/rest/testdata/rest_catalog/docker-compose.yaml deleted file mode 100644 index 5c10146..0000000 --- a/crates/rest/testdata/rest_catalog/docker-compose.yaml +++ /dev/null @@ -1,65 +0,0 @@ -# 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. - -version: '3.8' - -services: - rest: - image: tabulario/iceberg-rest:0.10.0 - environment: - - AWS_ACCESS_KEY_ID=admin - - AWS_SECRET_ACCESS_KEY=password - - AWS_REGION=us-east-1 - - CATALOG_CATOLOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog - - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory - - CATALOG_WAREHOUSE=s3://icebergdata/demo - - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO - - CATALOG_S3_ENDPOINT=http://minio:9000 - depends_on: - - minio - links: - - minio:icebergdata.minio - expose: - - 8181 - - minio: - image: minio/minio - environment: - - MINIO_ROOT_USER=admin - - MINIO_ROOT_PASSWORD=password - - MINIO_DOMAIN=minio - expose: - - 9001 - - 9000 - command: [ "server", "/data", "--console-address", ":9001" ] - - mc: - depends_on: - - minio - image: minio/mc - environment: - - AWS_ACCESS_KEY_ID=admin - - AWS_SECRET_ACCESS_KEY=password - - AWS_REGION=us-east-1 - entrypoint: > - /bin/sh -c " - until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done; - /usr/bin/mc rm -r --force minio/icebergdata; - /usr/bin/mc mb minio/icebergdata; - /usr/bin/mc policy set public minio/icebergdata; - tail -f /dev/null - " \ No newline at end of file diff --git a/crates/rest/testdata/update_table_response.json b/crates/rest/testdata/update_table_response.json deleted file mode 100644 index 80ec269..0000000 --- a/crates/rest/testdata/update_table_response.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "metadata-location": "s3://warehouse/database/table/metadata.json", - "metadata": { - "format-version": 2, - "table-uuid": "bf289591-dcc0-4234-ad4f-5c3eed811a29", - "location": "s3://warehouse/database/table", - "last-sequence-number" : 1, - "last-updated-ms": 1657810967051, - "last-column-id": 3, - "current-schema-id": 0, - "schemas": [ - { - "type": "struct", - "schema-id": 0, - "identifier-field-ids": [2], - "fields": [ - {"id": 1, "name": "foo", "required": false, "type": "string"}, - {"id": 2, "name": "bar", "required": true, "type": "int"}, - {"id": 3, "name": "baz", "required": false, "type": "boolean"} - ] - } - ], - "partition-specs": [], - "default-spec-id": 0, - "last-partition-id": 999, - "default-sort-order-id": 0, - "sort-orders": [{"order-id": 0, "fields": []}], - "properties": { - "write.delete.parquet.compression-codec": "zstd", - "write.metadata.compression-codec": "gzip", - "write.summary.partition-limit": "100", - "write.parquet.compression-codec": "zstd" - }, - "current-snapshot-id": -1, - "refs": {}, - "snapshots": [], - "snapshot-log": [], - "metadata-log": [] - } -} \ No newline at end of file diff --git a/crates/rest/tests/rest_catalog_test.rs b/crates/rest/tests/rest_catalog_test.rs deleted file mode 100644 index a4d0795..0000000 --- a/crates/rest/tests/rest_catalog_test.rs +++ /dev/null @@ -1,376 +0,0 @@ -// 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. - -//! Integration tests for rest catalog. - -use iceberg::spec::{FormatVersion, NestedField, PrimitiveType, Schema, Type}; -use iceberg::transaction::Transaction; -use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; -use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; -use iceberg_test_utils::docker::DockerCompose; -use iceberg_test_utils::{normalize_test_name, set_up}; -use port_scanner::scan_port_addr; -use std::collections::HashMap; -use tokio::time::sleep; - -const REST_CATALOG_PORT: u16 = 8181; - -struct TestFixture { - _docker_compose: DockerCompose, - rest_catalog: RestCatalog, -} - -async fn set_test_fixture(func: &str) -> TestFixture { - set_up(); - let docker_compose = DockerCompose::new( - normalize_test_name(format!("{}_{func}", module_path!())), - format!("{}/testdata/rest_catalog", env!("CARGO_MANIFEST_DIR")), - ); - - // Start docker compose - docker_compose.run(); - - let rest_catalog_ip = docker_compose.get_container_ip("rest"); - - let read_port = format!("{}:{}", rest_catalog_ip, REST_CATALOG_PORT); - loop { - if !scan_port_addr(&read_port) { - log::info!("Waiting for 1s rest catalog to ready..."); - sleep(std::time::Duration::from_millis(1000)).await; - } else { - break; - } - } - - let config = RestCatalogConfig::builder() - .uri(format!("http://{}:{}", rest_catalog_ip, REST_CATALOG_PORT)) - .build(); - let rest_catalog = RestCatalog::new(config).await.unwrap(); - - TestFixture { - _docker_compose: docker_compose, - rest_catalog, - } -} -#[tokio::test] -async fn test_get_non_exist_namespace() { - let fixture = set_test_fixture("test_get_non_exist_namespace").await; - - let result = fixture - .rest_catalog - .get_namespace(&NamespaceIdent::from_strs(["demo"]).unwrap()) - .await; - - assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains("Namespace does not exist")); -} - -#[tokio::test] -async fn test_get_namespace() { - let fixture = set_test_fixture("test_get_namespace").await; - - let ns = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Verify that namespace doesn't exist - assert!(fixture.rest_catalog.get_namespace(ns.name()).await.is_err()); - - // Create this namespace - let created_ns = fixture - .rest_catalog - .create_namespace(ns.name(), ns.properties().clone()) - .await - .unwrap(); - - assert_eq!(ns.name(), created_ns.name()); - assert_map_contains(ns.properties(), created_ns.properties()); - - // Check that this namespace already exists - let get_ns = fixture.rest_catalog.get_namespace(ns.name()).await.unwrap(); - assert_eq!(ns.name(), get_ns.name()); - assert_map_contains(ns.properties(), created_ns.properties()); -} - -#[tokio::test] -async fn test_list_namespace() { - let fixture = set_test_fixture("test_list_namespace").await; - - let ns1 = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - let ns2 = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "macos"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "xuanwo".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Currently this namespace doesn't exist, so it should return error. - assert!(fixture - .rest_catalog - .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) - .await - .is_err()); - - // Create namespaces - fixture - .rest_catalog - .create_namespace(ns1.name(), ns1.properties().clone()) - .await - .unwrap(); - fixture - .rest_catalog - .create_namespace(ns2.name(), ns1.properties().clone()) - .await - .unwrap(); - - // List namespace - let mut nss = fixture - .rest_catalog - .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) - .await - .unwrap(); - nss.sort(); - - assert_eq!(&nss[0], ns1.name()); - assert_eq!(&nss[1], ns2.name()); -} - -#[tokio::test] -async fn test_list_empty_namespace() { - let fixture = set_test_fixture("test_list_empty_namespace").await; - - let ns_apple = Namespace::with_properties( - NamespaceIdent::from_strs(["apple"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Currently this namespace doesn't exist, so it should return error. - assert!(fixture - .rest_catalog - .list_namespaces(Some(ns_apple.name())) - .await - .is_err()); - - // Create namespaces - fixture - .rest_catalog - .create_namespace(ns_apple.name(), ns_apple.properties().clone()) - .await - .unwrap(); - - // List namespace - let nss = fixture - .rest_catalog - .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) - .await - .unwrap(); - assert!(nss.is_empty()); -} - -#[tokio::test] -async fn test_list_root_namespace() { - let fixture = set_test_fixture("test_list_root_namespace").await; - - let ns1 = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - let ns2 = Namespace::with_properties( - NamespaceIdent::from_strs(["google", "android"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "xuanwo".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Currently this namespace doesn't exist, so it should return error. - assert!(fixture - .rest_catalog - .list_namespaces(Some(&NamespaceIdent::from_strs(["apple"]).unwrap())) - .await - .is_err()); - - // Create namespaces - fixture - .rest_catalog - .create_namespace(ns1.name(), ns1.properties().clone()) - .await - .unwrap(); - fixture - .rest_catalog - .create_namespace(ns2.name(), ns1.properties().clone()) - .await - .unwrap(); - - // List namespace - let mut nss = fixture.rest_catalog.list_namespaces(None).await.unwrap(); - nss.sort(); - - assert_eq!(&nss[0], &NamespaceIdent::from_strs(["apple"]).unwrap()); - assert_eq!(&nss[1], &NamespaceIdent::from_strs(["google"]).unwrap()); -} - -#[tokio::test] -async fn test_create_table() { - let fixture = set_test_fixture("test_create_table").await; - - let ns = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Create namespaces - fixture - .rest_catalog - .create_namespace(ns.name(), ns.properties().clone()) - .await - .unwrap(); - - let schema = Schema::builder() - .with_schema_id(1) - .with_identifier_field_ids(vec![2]) - .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(), - ]) - .build() - .unwrap(); - - let table_creation = TableCreation::builder() - .name("t1".to_string()) - .schema(schema.clone()) - .build(); - - let table = fixture - .rest_catalog - .create_table(ns.name(), table_creation) - .await - .unwrap(); - - assert_eq!( - table.identifier(), - &TableIdent::new(ns.name().clone(), "t1".to_string()) - ); - - assert_eq!( - table.metadata().current_schema().as_struct(), - schema.as_struct() - ); - assert_eq!(table.metadata().format_version(), FormatVersion::V2); - assert!(table.metadata().current_snapshot().is_none()); - assert!(table.metadata().history().is_empty()); - assert!(table.metadata().default_sort_order().unwrap().is_unsorted()); - assert!(table - .metadata() - .default_partition_spec() - .unwrap() - .is_unpartitioned()); -} - -#[tokio::test] -async fn test_update_table() { - let fixture = set_test_fixture("test_update_table").await; - - let ns = Namespace::with_properties( - NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), - HashMap::from([ - ("owner".to_string(), "ray".to_string()), - ("community".to_string(), "apache".to_string()), - ]), - ); - - // Create namespaces - fixture - .rest_catalog - .create_namespace(ns.name(), ns.properties().clone()) - .await - .unwrap(); - - let schema = Schema::builder() - .with_schema_id(1) - .with_identifier_field_ids(vec![2]) - .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(), - ]) - .build() - .unwrap(); - - // Now we create a table - let table_creation = TableCreation::builder() - .name("t1".to_string()) - .schema(schema.clone()) - .build(); - - let table = fixture - .rest_catalog - .create_table(ns.name(), table_creation) - .await - .unwrap(); - - assert_eq!( - table.identifier(), - &TableIdent::new(ns.name().clone(), "t1".to_string()) - ); - - // Update table by committing transaction - let table2 = Transaction::new(&table) - .set_properties(HashMap::from([("prop1".to_string(), "v1".to_string())])) - .unwrap() - .commit(&fixture.rest_catalog) - .await - .unwrap(); - - assert_map_contains( - &HashMap::from([("prop1".to_string(), "v1".to_string())]), - table2.metadata().properties(), - ); -} - -fn assert_map_contains(map1: &HashMap, map2: &HashMap) { - for (k, v) in map1 { - assert!(map2.contains_key(k)); - assert_eq!(map2.get(k).unwrap(), v); - } -} diff --git a/crates/test_utils/Cargo.toml b/crates/test_utils/Cargo.toml deleted file mode 100644 index 2ad7ef9..0000000 --- a/crates/test_utils/Cargo.toml +++ /dev/null @@ -1,31 +0,0 @@ -# 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_test_utils" -version = { workspace = true } -edition = { workspace = true } -rust-version = { workspace = true } -repository = { workspace = true } -license = { workspace = true } - -[dependencies] -env_logger = { workspace = true } -log = "0.4.20" - -[features] -tests = [] diff --git a/crates/test_utils/src/cmd.rs b/crates/test_utils/src/cmd.rs deleted file mode 100644 index 604d4a1..0000000 --- a/crates/test_utils/src/cmd.rs +++ /dev/null @@ -1,41 +0,0 @@ -// 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::process::Command; - -pub fn run_command(mut cmd: Command, desc: impl ToString) { - let desc = desc.to_string(); - log::info!("Starting to {}, command: {:?}", &desc, cmd); - let exit = cmd.status().unwrap(); - if exit.success() { - log::info!("{} succeed!", desc) - } else { - panic!("{} failed: {:?}", desc, exit); - } -} - -pub fn get_cmd_output(mut cmd: Command, desc: impl ToString) -> String { - let desc = desc.to_string(); - log::info!("Starting to {}, command: {:?}", &desc, cmd); - let output = cmd.output().unwrap(); - if output.status.success() { - log::info!("{} succeed!", desc); - String::from_utf8(output.stdout).unwrap() - } else { - panic!("{} failed: {:?}", desc, output.status); - } -} diff --git a/crates/test_utils/src/docker.rs b/crates/test_utils/src/docker.rs deleted file mode 100644 index 6c5fbef..0000000 --- a/crates/test_utils/src/docker.rs +++ /dev/null @@ -1,102 +0,0 @@ -// 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 crate::cmd::{get_cmd_output, run_command}; -use std::process::Command; - -/// A utility to manage lifecycle of docker compose. -/// -/// It's will start docker compose when calling `run` method, and will be stopped when dropped. -#[derive(Debug)] -pub struct DockerCompose { - project_name: String, - docker_compose_dir: String, -} - -impl DockerCompose { - pub fn new(project_name: impl ToString, docker_compose_dir: impl ToString) -> Self { - Self { - project_name: project_name.to_string(), - docker_compose_dir: docker_compose_dir.to_string(), - } - } - - pub fn project_name(&self) -> &str { - self.project_name.as_str() - } - - pub fn run(&self) { - let mut cmd = Command::new("docker"); - cmd.current_dir(&self.docker_compose_dir); - - cmd.args(vec![ - "compose", - "-p", - self.project_name.as_str(), - "up", - "-d", - "--wait", - "--timeout", - "1200000", - ]); - - run_command( - cmd, - format!( - "Starting docker compose in {}, project name: {}", - self.docker_compose_dir, self.project_name - ), - ) - } - - pub fn get_container_ip(&self, service_name: impl AsRef) -> String { - let container_name = format!("{}-{}-1", self.project_name, service_name.as_ref()); - let mut cmd = Command::new("docker"); - cmd.arg("inspect") - .arg("-f") - .arg("{{range.NetworkSettings.Networks}}{{.IPAddress}}{{end}}") - .arg(&container_name); - - get_cmd_output(cmd, format!("Get container ip of {container_name}")) - .trim() - .to_string() - } -} - -impl Drop for DockerCompose { - fn drop(&mut self) { - let mut cmd = Command::new("docker"); - cmd.current_dir(&self.docker_compose_dir); - - cmd.args(vec![ - "compose", - "-p", - self.project_name.as_str(), - "down", - "-v", - "--remove-orphans", - ]); - - run_command( - cmd, - format!( - "Stopping docker compose in {}, project name: {}", - self.docker_compose_dir, self.project_name - ), - ) - } -} diff --git a/crates/test_utils/src/lib.rs b/crates/test_utils/src/lib.rs deleted file mode 100644 index 4f63b8d..0000000 --- a/crates/test_utils/src/lib.rs +++ /dev/null @@ -1,41 +0,0 @@ -// 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. - -//! This crate contains common utilities for testing. -//! -//! It's not intended for use outside of `iceberg-rust`. - -#[cfg(feature = "tests")] -mod cmd; -#[cfg(feature = "tests")] -pub mod docker; - -#[cfg(feature = "tests")] -pub use common::*; - -#[cfg(feature = "tests")] -mod common { - use std::sync::Once; - - static INIT: Once = Once::new(); - pub fn set_up() { - INIT.call_once(env_logger::init); - } - pub fn normalize_test_name(s: impl ToString) -> String { - s.to_string().replace("::", "__").replace('.', "_") - } -} diff --git a/rustfmt.toml b/rustfmt.toml new file mode 100644 index 0000000..54aa0cc --- /dev/null +++ b/rustfmt.toml @@ -0,0 +1,4 @@ +edition = "2018" +max_width = 100 +tab_spaces = 2 +hard_tabs = false diff --git a/scripts/parse_dependencies.py b/scripts/parse_dependencies.py new file mode 100644 index 0000000..551f3a7 --- /dev/null +++ b/scripts/parse_dependencies.py @@ -0,0 +1,42 @@ +import os +import sys + +begin = False +package_version = {} +with open('./Cargo.toml') as f: + for line in f: + if '[' == line[0]: + begin = False + if 'dependencies' in line: + begin = True + continue + + if begin: + sep = line.find('=') + package_version[line[:sep-1].strip()] = line[sep+2:].strip() + +for dir_path in ["./libs/iceberg/", "./libs/rest/", "./libs/test_utils/"]: + r = open(dir_path + "Cargo.toml") + w = open(dir_path + "Cargo_n.toml", 'w') + begin = False + for r_line in r: + if '[' == r_line[0]: + begin = False + if 'dependencies' in r_line: + begin = True + w.write(r_line) + continue + + if begin: + sep = r_line.find('=') + package = r_line[:sep-1].strip() + if package in package_version: + w.writelines([f"{package} = {package_version[package]}", "\n"]) + else: + w.write(r_line) + else: + w.write(r_line) + r.close() + w.close() + os.remove(dir_path + "Cargo.toml") + os.rename(dir_path + "Cargo_n.toml", dir_path + "Cargo.toml") diff --git a/src/main.rs b/src/main.rs index e7a11a9..7e105ac 100644 --- a/src/main.rs +++ b/src/main.rs @@ -1,3 +1,30 @@ -fn main() { - println!("Hello, world!"); +#[macro_use] +extern crate rocket; + +mod server; +use server::routes::*; + +#[launch] +fn rocket() -> _ { + rocket::build().mount( + "/v1", + routes![ + namespace::get_namespace, + namespace::post_namespace, + namespace::head_namespace_by_name, + namespace::get_namespace_by_name, + namespace::delete_namespace_by_name, + namespace::post_namespace_properties, + table::get_table_by_namespace, + table::post_table_by_namespace, + table::register_table, + table::get_table, + table::post_table, + table::delete_table, + table::head_table, + table::rename_table, + metric::post_metrics, + config::get_config, + ], + ) } diff --git a/src/server/mod.rs b/src/server/mod.rs new file mode 100644 index 0000000..6a664ab --- /dev/null +++ b/src/server/mod.rs @@ -0,0 +1 @@ +pub mod routes; diff --git a/src/server/routes/config.rs b/src/server/routes/config.rs new file mode 100644 index 0000000..22e875d --- /dev/null +++ b/src/server/routes/config.rs @@ -0,0 +1,4 @@ +#[get("/config")] +pub fn get_config() { + todo!("get_config") +} diff --git a/src/server/routes/metric.rs b/src/server/routes/metric.rs new file mode 100644 index 0000000..1c50f22 --- /dev/null +++ b/src/server/routes/metric.rs @@ -0,0 +1,5 @@ +/// Send a metrics report to this endpoint to be processed by the backend +#[post("/namespaces//tables/
/metrics")] +pub fn post_metrics(namespace: &str, table: &str) { + todo!("post_metrics") +} diff --git a/src/server/routes/mod.rs b/src/server/routes/mod.rs new file mode 100644 index 0000000..d93f96d --- /dev/null +++ b/src/server/routes/mod.rs @@ -0,0 +1,5 @@ +#[allow(dead_code)] +pub mod config; +pub mod metric; +pub mod namespace; +pub mod table; diff --git a/src/server/routes/namespace.rs b/src/server/routes/namespace.rs new file mode 100644 index 0000000..6c08ff0 --- /dev/null +++ b/src/server/routes/namespace.rs @@ -0,0 +1,35 @@ +/// List namespaces, optionally providing a parent namespace to list underneath +#[get("/namespaces")] +pub fn get_namespace() { + todo!("get_namespace") +} + +/// Create a namespace +#[post("/namespaces")] +pub fn post_namespace() { + todo!("post_namespace") +} + +/// Check if a namespace exists +#[head("/namespaces/")] +pub fn head_namespace_by_name(namespace: &str) { + todo!("head_namespace_by_name") +} + +/// Load the metadata properties for a namespace +#[get("/namespaces/")] +pub fn get_namespace_by_name(namespace: &str) { + todo!("get_namespace_by_name") +} + +/// Drop a namespace from the catalog. Namespace must be empty. +#[delete("/namespaces/")] +pub fn delete_namespace_by_name(namespace: &str) { + todo!("delete_namespace_by_name") +} + +/// Set or remove properties on a namespace +#[post("/namespaces//properties")] +pub fn post_namespace_properties(namespace: &str) { + todo!("post_namespace_properties") +} diff --git a/src/server/routes/table.rs b/src/server/routes/table.rs new file mode 100644 index 0000000..a68e9c3 --- /dev/null +++ b/src/server/routes/table.rs @@ -0,0 +1,47 @@ +/// List all table identifiers underneath a given namespace +#[get("/namespaces//tables")] +pub fn get_table_by_namespace(namespace: &str) { + todo!("get_table_by_namespace") +} + +/// Create a table in the given namespace +#[post("/namespaces//tables")] +pub fn post_table_by_namespace(namespace: &str) { + todo!("post_table_by_namespace") +} + +/// Register a table in the given namespace using given metadata file location +#[post("/namespaces//register")] +pub fn register_table(namespace: &str) { + todo!("register_table") +} + +/// Load a table from the catalog +#[get("/namespaces//tables/
")] +pub fn get_table(namespace: &str, table: &str) { + todo!("post_namespace_table") +} + +/// Commit updates to a table +#[post("/namespaces//tables/
")] +pub fn post_table(namespace: &str, table: &str) { + todo!("post_namespace_table") +} + +/// Drop a table from the catalog +#[delete("/namespaces//tables/
")] +pub fn delete_table(namespace: &str, table: &str) { + todo!("post_namespace_table") +} + +/// Check if a table exists +#[head("/namespaces//tables/
")] +pub fn head_table(namespace: &str, table: &str) { + todo!("post_namespace_table") +} + +/// Rename a table from its current name to a new name +#[post("/tables/rename")] +pub fn rename_table() { + todo!("rename_table") +} From f15b3690f6370563e014af4628b7897afa49a799 Mon Sep 17 00:00:00 2001 From: zhouzilong <529620861@qq.com> Date: Tue, 27 Feb 2024 11:55:44 -0500 Subject: [PATCH 3/4] rename crates to libs --- crates/examples/Cargo.toml | 37 ---------- crates/examples/README.md | 21 ------ crates/examples/src/rest_catalog_namespace.rs | 53 --------------- crates/examples/src/rest_catalog_table.rs | 68 ------------------- {crates => libs}/iceberg/Cargo.toml | 0 .../iceberg/DEPENDENCIES.rust.tsv | 0 {crates => libs}/iceberg/src/avro/mod.rs | 0 {crates => libs}/iceberg/src/avro/schema.rs | 0 {crates => libs}/iceberg/src/catalog/mod.rs | 0 {crates => libs}/iceberg/src/error.rs | 0 {crates => libs}/iceberg/src/expr/mod.rs | 0 .../iceberg/src/expr/predicate.rs | 0 {crates => libs}/iceberg/src/expr/term.rs | 0 {crates => libs}/iceberg/src/io.rs | 0 {crates => libs}/iceberg/src/lib.rs | 0 {crates => libs}/iceberg/src/scan.rs | 0 .../iceberg/src/spec/datatypes.rs | 0 {crates => libs}/iceberg/src/spec/manifest.rs | 0 .../iceberg/src/spec/manifest_list.rs | 0 {crates => libs}/iceberg/src/spec/mod.rs | 0 .../iceberg/src/spec/partition.rs | 0 {crates => libs}/iceberg/src/spec/schema.rs | 0 {crates => libs}/iceberg/src/spec/snapshot.rs | 0 {crates => libs}/iceberg/src/spec/sort.rs | 0 .../iceberg/src/spec/table_metadata.rs | 0 .../iceberg/src/spec/transform.rs | 0 {crates => libs}/iceberg/src/spec/values.rs | 0 {crates => libs}/iceberg/src/table.rs | 0 {crates => libs}/iceberg/src/transaction.rs | 0 .../iceberg/src/transform/bucket.rs | 0 .../iceberg/src/transform/identity.rs | 0 {crates => libs}/iceberg/src/transform/mod.rs | 0 .../iceberg/src/transform/temporal.rs | 0 .../iceberg/src/transform/truncate.rs | 0 .../iceberg/src/transform/void.rs | 0 .../iceberg/src/writer/file_writer/mod.rs | 0 {crates => libs}/iceberg/src/writer/mod.rs | 0 .../testdata/avro_schema_manifest_entry.json | 0 .../avro_schema_manifest_file_v1.json | 0 .../avro_schema_manifest_file_v2.json | 0 .../testdata/example_table_metadata_v2.json | 0 .../TableMetadataUnsupportedVersion.json | 0 .../table_metadata/TableMetadataV1Valid.json | 0 .../TableMetadataV2CurrentSchemaNotFound.json | 0 ...TableMetadataV2MissingLastPartitionId.json | 0 .../TableMetadataV2MissingPartitionSpecs.json | 0 .../TableMetadataV2MissingSchemas.json | 0 .../TableMetadataV2MissingSortOrder.json | 0 .../table_metadata/TableMetadataV2Valid.json | 0 .../TableMetadataV2ValidMinimal.json | 0 50 files changed, 179 deletions(-) delete mode 100644 crates/examples/Cargo.toml delete mode 100644 crates/examples/README.md delete mode 100644 crates/examples/src/rest_catalog_namespace.rs delete mode 100644 crates/examples/src/rest_catalog_table.rs rename {crates => libs}/iceberg/Cargo.toml (100%) rename {crates => libs}/iceberg/DEPENDENCIES.rust.tsv (100%) rename {crates => libs}/iceberg/src/avro/mod.rs (100%) rename {crates => libs}/iceberg/src/avro/schema.rs (100%) rename {crates => libs}/iceberg/src/catalog/mod.rs (100%) rename {crates => libs}/iceberg/src/error.rs (100%) rename {crates => libs}/iceberg/src/expr/mod.rs (100%) rename {crates => libs}/iceberg/src/expr/predicate.rs (100%) rename {crates => libs}/iceberg/src/expr/term.rs (100%) rename {crates => libs}/iceberg/src/io.rs (100%) rename {crates => libs}/iceberg/src/lib.rs (100%) rename {crates => libs}/iceberg/src/scan.rs (100%) rename {crates => libs}/iceberg/src/spec/datatypes.rs (100%) rename {crates => libs}/iceberg/src/spec/manifest.rs (100%) rename {crates => libs}/iceberg/src/spec/manifest_list.rs (100%) rename {crates => libs}/iceberg/src/spec/mod.rs (100%) rename {crates => libs}/iceberg/src/spec/partition.rs (100%) rename {crates => libs}/iceberg/src/spec/schema.rs (100%) rename {crates => libs}/iceberg/src/spec/snapshot.rs (100%) rename {crates => libs}/iceberg/src/spec/sort.rs (100%) rename {crates => libs}/iceberg/src/spec/table_metadata.rs (100%) rename {crates => libs}/iceberg/src/spec/transform.rs (100%) rename {crates => libs}/iceberg/src/spec/values.rs (100%) rename {crates => libs}/iceberg/src/table.rs (100%) rename {crates => libs}/iceberg/src/transaction.rs (100%) rename {crates => libs}/iceberg/src/transform/bucket.rs (100%) rename {crates => libs}/iceberg/src/transform/identity.rs (100%) rename {crates => libs}/iceberg/src/transform/mod.rs (100%) rename {crates => libs}/iceberg/src/transform/temporal.rs (100%) rename {crates => libs}/iceberg/src/transform/truncate.rs (100%) rename {crates => libs}/iceberg/src/transform/void.rs (100%) rename {crates => libs}/iceberg/src/writer/file_writer/mod.rs (100%) rename {crates => libs}/iceberg/src/writer/mod.rs (100%) rename {crates => libs}/iceberg/testdata/avro_schema_manifest_entry.json (100%) rename {crates => libs}/iceberg/testdata/avro_schema_manifest_file_v1.json (100%) rename {crates => libs}/iceberg/testdata/avro_schema_manifest_file_v2.json (100%) rename {crates => libs}/iceberg/testdata/example_table_metadata_v2.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV1Valid.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2Valid.json (100%) rename {crates => libs}/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json (100%) diff --git a/crates/examples/Cargo.toml b/crates/examples/Cargo.toml deleted file mode 100644 index 57596c2..0000000 --- a/crates/examples/Cargo.toml +++ /dev/null @@ -1,37 +0,0 @@ -# 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-examples" -version = { workspace = true } -edition = { workspace = true } -repository = { workspace = true } -license = { workspace = true } -rust-version = { workspace = true } - -[dependencies] -iceberg = { workspace = true } -iceberg-catalog-rest = { workspace = true } -tokio = { version = "1", features = ["full"] } - -[[example]] -name = "rest-catalog-namespace" -path = "src/rest_catalog_namespace.rs" - -[[example]] -name = "rest-catalog-table" -path = "src/rest_catalog_table.rs" diff --git a/crates/examples/README.md b/crates/examples/README.md deleted file mode 100644 index 335d2ea..0000000 --- a/crates/examples/README.md +++ /dev/null @@ -1,21 +0,0 @@ - - -Example usage codes for `iceberg-rust`. Currently, these examples can't run directly since it requires setting up of -environments for catalogs, for example, rest catalog server. \ No newline at end of file diff --git a/crates/examples/src/rest_catalog_namespace.rs b/crates/examples/src/rest_catalog_namespace.rs deleted file mode 100644 index 0a3b00b..0000000 --- a/crates/examples/src/rest_catalog_namespace.rs +++ /dev/null @@ -1,53 +0,0 @@ -// 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 iceberg::{Catalog, NamespaceIdent}; -use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; -use std::collections::HashMap; - -#[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).await.unwrap(); - // ANCHOR_END: create_catalog - - // ANCHOR: list_all_namespace - // List all namespaces - let all_namespaces = catalog.list_namespaces(None).await.unwrap(); - println!("Namespaces in current catalog: {:?}", all_namespaces); - // ANCHOR_END: list_all_namespace - - // ANCHOR: create_namespace - let namespace_id = - NamespaceIdent::from_vec(vec!["ns1".to_string(), "ns11".to_string()]).unwrap(); - // Create namespace - let ns = catalog - .create_namespace( - &namespace_id, - HashMap::from([("key1".to_string(), "value1".to_string())]), - ) - .await - .unwrap(); - - println!("Namespace created: {:?}", ns); - // ANCHOR_END: create_namespace -} diff --git a/crates/examples/src/rest_catalog_table.rs b/crates/examples/src/rest_catalog_table.rs deleted file mode 100644 index 9fb3dd7..0000000 --- a/crates/examples/src/rest_catalog_table.rs +++ /dev/null @@ -1,68 +0,0 @@ -// 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 iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; -use iceberg::{Catalog, TableCreation, TableIdent}; -use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; -use std::collections::HashMap; - -#[tokio::main] -async fn main() { - // Create catalog - let config = RestCatalogConfig::builder() - .uri("http://localhost:8080".to_string()) - .build(); - - let catalog = RestCatalog::new(config).await.unwrap(); - - // ANCHOR: create_table - let table_id = TableIdent::from_strs(["default", "t1"]).unwrap(); - - 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(), - ]) - .with_schema_id(1) - .with_identifier_field_ids(vec![2]) - .build() - .unwrap(); - - // Create table - let table_creation = TableCreation::builder() - .name(table_id.name.clone()) - .schema(table_schema.clone()) - .properties(HashMap::from([("owner".to_string(), "testx".to_string())])) - .build(); - - 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(); - println!("{:?}", table2.metadata()); - // ANCHOR_END: load_table -} diff --git a/crates/iceberg/Cargo.toml b/libs/iceberg/Cargo.toml similarity index 100% rename from crates/iceberg/Cargo.toml rename to libs/iceberg/Cargo.toml diff --git a/crates/iceberg/DEPENDENCIES.rust.tsv b/libs/iceberg/DEPENDENCIES.rust.tsv similarity index 100% rename from crates/iceberg/DEPENDENCIES.rust.tsv rename to libs/iceberg/DEPENDENCIES.rust.tsv diff --git a/crates/iceberg/src/avro/mod.rs b/libs/iceberg/src/avro/mod.rs similarity index 100% rename from crates/iceberg/src/avro/mod.rs rename to libs/iceberg/src/avro/mod.rs diff --git a/crates/iceberg/src/avro/schema.rs b/libs/iceberg/src/avro/schema.rs similarity index 100% rename from crates/iceberg/src/avro/schema.rs rename to libs/iceberg/src/avro/schema.rs diff --git a/crates/iceberg/src/catalog/mod.rs b/libs/iceberg/src/catalog/mod.rs similarity index 100% rename from crates/iceberg/src/catalog/mod.rs rename to libs/iceberg/src/catalog/mod.rs diff --git a/crates/iceberg/src/error.rs b/libs/iceberg/src/error.rs similarity index 100% rename from crates/iceberg/src/error.rs rename to libs/iceberg/src/error.rs diff --git a/crates/iceberg/src/expr/mod.rs b/libs/iceberg/src/expr/mod.rs similarity index 100% rename from crates/iceberg/src/expr/mod.rs rename to libs/iceberg/src/expr/mod.rs diff --git a/crates/iceberg/src/expr/predicate.rs b/libs/iceberg/src/expr/predicate.rs similarity index 100% rename from crates/iceberg/src/expr/predicate.rs rename to libs/iceberg/src/expr/predicate.rs diff --git a/crates/iceberg/src/expr/term.rs b/libs/iceberg/src/expr/term.rs similarity index 100% rename from crates/iceberg/src/expr/term.rs rename to libs/iceberg/src/expr/term.rs diff --git a/crates/iceberg/src/io.rs b/libs/iceberg/src/io.rs similarity index 100% rename from crates/iceberg/src/io.rs rename to libs/iceberg/src/io.rs diff --git a/crates/iceberg/src/lib.rs b/libs/iceberg/src/lib.rs similarity index 100% rename from crates/iceberg/src/lib.rs rename to libs/iceberg/src/lib.rs diff --git a/crates/iceberg/src/scan.rs b/libs/iceberg/src/scan.rs similarity index 100% rename from crates/iceberg/src/scan.rs rename to libs/iceberg/src/scan.rs diff --git a/crates/iceberg/src/spec/datatypes.rs b/libs/iceberg/src/spec/datatypes.rs similarity index 100% rename from crates/iceberg/src/spec/datatypes.rs rename to libs/iceberg/src/spec/datatypes.rs diff --git a/crates/iceberg/src/spec/manifest.rs b/libs/iceberg/src/spec/manifest.rs similarity index 100% rename from crates/iceberg/src/spec/manifest.rs rename to libs/iceberg/src/spec/manifest.rs diff --git a/crates/iceberg/src/spec/manifest_list.rs b/libs/iceberg/src/spec/manifest_list.rs similarity index 100% rename from crates/iceberg/src/spec/manifest_list.rs rename to libs/iceberg/src/spec/manifest_list.rs diff --git a/crates/iceberg/src/spec/mod.rs b/libs/iceberg/src/spec/mod.rs similarity index 100% rename from crates/iceberg/src/spec/mod.rs rename to libs/iceberg/src/spec/mod.rs diff --git a/crates/iceberg/src/spec/partition.rs b/libs/iceberg/src/spec/partition.rs similarity index 100% rename from crates/iceberg/src/spec/partition.rs rename to libs/iceberg/src/spec/partition.rs diff --git a/crates/iceberg/src/spec/schema.rs b/libs/iceberg/src/spec/schema.rs similarity index 100% rename from crates/iceberg/src/spec/schema.rs rename to libs/iceberg/src/spec/schema.rs diff --git a/crates/iceberg/src/spec/snapshot.rs b/libs/iceberg/src/spec/snapshot.rs similarity index 100% rename from crates/iceberg/src/spec/snapshot.rs rename to libs/iceberg/src/spec/snapshot.rs diff --git a/crates/iceberg/src/spec/sort.rs b/libs/iceberg/src/spec/sort.rs similarity index 100% rename from crates/iceberg/src/spec/sort.rs rename to libs/iceberg/src/spec/sort.rs diff --git a/crates/iceberg/src/spec/table_metadata.rs b/libs/iceberg/src/spec/table_metadata.rs similarity index 100% rename from crates/iceberg/src/spec/table_metadata.rs rename to libs/iceberg/src/spec/table_metadata.rs diff --git a/crates/iceberg/src/spec/transform.rs b/libs/iceberg/src/spec/transform.rs similarity index 100% rename from crates/iceberg/src/spec/transform.rs rename to libs/iceberg/src/spec/transform.rs diff --git a/crates/iceberg/src/spec/values.rs b/libs/iceberg/src/spec/values.rs similarity index 100% rename from crates/iceberg/src/spec/values.rs rename to libs/iceberg/src/spec/values.rs diff --git a/crates/iceberg/src/table.rs b/libs/iceberg/src/table.rs similarity index 100% rename from crates/iceberg/src/table.rs rename to libs/iceberg/src/table.rs diff --git a/crates/iceberg/src/transaction.rs b/libs/iceberg/src/transaction.rs similarity index 100% rename from crates/iceberg/src/transaction.rs rename to libs/iceberg/src/transaction.rs diff --git a/crates/iceberg/src/transform/bucket.rs b/libs/iceberg/src/transform/bucket.rs similarity index 100% rename from crates/iceberg/src/transform/bucket.rs rename to libs/iceberg/src/transform/bucket.rs diff --git a/crates/iceberg/src/transform/identity.rs b/libs/iceberg/src/transform/identity.rs similarity index 100% rename from crates/iceberg/src/transform/identity.rs rename to libs/iceberg/src/transform/identity.rs diff --git a/crates/iceberg/src/transform/mod.rs b/libs/iceberg/src/transform/mod.rs similarity index 100% rename from crates/iceberg/src/transform/mod.rs rename to libs/iceberg/src/transform/mod.rs diff --git a/crates/iceberg/src/transform/temporal.rs b/libs/iceberg/src/transform/temporal.rs similarity index 100% rename from crates/iceberg/src/transform/temporal.rs rename to libs/iceberg/src/transform/temporal.rs diff --git a/crates/iceberg/src/transform/truncate.rs b/libs/iceberg/src/transform/truncate.rs similarity index 100% rename from crates/iceberg/src/transform/truncate.rs rename to libs/iceberg/src/transform/truncate.rs diff --git a/crates/iceberg/src/transform/void.rs b/libs/iceberg/src/transform/void.rs similarity index 100% rename from crates/iceberg/src/transform/void.rs rename to libs/iceberg/src/transform/void.rs diff --git a/crates/iceberg/src/writer/file_writer/mod.rs b/libs/iceberg/src/writer/file_writer/mod.rs similarity index 100% rename from crates/iceberg/src/writer/file_writer/mod.rs rename to libs/iceberg/src/writer/file_writer/mod.rs diff --git a/crates/iceberg/src/writer/mod.rs b/libs/iceberg/src/writer/mod.rs similarity index 100% rename from crates/iceberg/src/writer/mod.rs rename to libs/iceberg/src/writer/mod.rs diff --git a/crates/iceberg/testdata/avro_schema_manifest_entry.json b/libs/iceberg/testdata/avro_schema_manifest_entry.json similarity index 100% rename from crates/iceberg/testdata/avro_schema_manifest_entry.json rename to libs/iceberg/testdata/avro_schema_manifest_entry.json diff --git a/crates/iceberg/testdata/avro_schema_manifest_file_v1.json b/libs/iceberg/testdata/avro_schema_manifest_file_v1.json similarity index 100% rename from crates/iceberg/testdata/avro_schema_manifest_file_v1.json rename to libs/iceberg/testdata/avro_schema_manifest_file_v1.json diff --git a/crates/iceberg/testdata/avro_schema_manifest_file_v2.json b/libs/iceberg/testdata/avro_schema_manifest_file_v2.json similarity index 100% rename from crates/iceberg/testdata/avro_schema_manifest_file_v2.json rename to libs/iceberg/testdata/avro_schema_manifest_file_v2.json diff --git a/crates/iceberg/testdata/example_table_metadata_v2.json b/libs/iceberg/testdata/example_table_metadata_v2.json similarity index 100% rename from crates/iceberg/testdata/example_table_metadata_v2.json rename to libs/iceberg/testdata/example_table_metadata_v2.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json b/libs/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json rename to libs/iceberg/testdata/table_metadata/TableMetadataUnsupportedVersion.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json b/libs/iceberg/testdata/table_metadata/TableMetadataV1Valid.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV1Valid.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV1Valid.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2CurrentSchemaNotFound.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2MissingLastPartitionId.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2MissingPartitionSpecs.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2MissingSchemas.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2MissingSortOrder.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2Valid.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2Valid.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2Valid.json diff --git a/crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json b/libs/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json similarity index 100% rename from crates/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json rename to libs/iceberg/testdata/table_metadata/TableMetadataV2ValidMinimal.json From 08ee3eb24e28b1b69087fae98913f17504dc787f Mon Sep 17 00:00:00 2001 From: zhouzilong <529620861@qq.com> Date: Tue, 27 Feb 2024 13:02:57 -0500 Subject: [PATCH 4/4] init db --- Cargo.toml | 4 +++- src/db/mod.rs | 5 +++++ 2 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 src/db/mod.rs diff --git a/Cargo.toml b/Cargo.toml index 67d32d9..6c16e15 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,4 +28,6 @@ rust-version = "1.75.0" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] rocket = { version = "0.5.0", features = ["json", "http2"] } -dotenv = "0.15.0" \ No newline at end of file +iceberg = { src = "./libs/iceberg" } +dotenv = "0.15.0" +pickledb = "^0.5.0" \ No newline at end of file diff --git a/src/db/mod.rs b/src/db/mod.rs new file mode 100644 index 0000000..2842023 --- /dev/null +++ b/src/db/mod.rs @@ -0,0 +1,5 @@ +use pickledb::PickleDb; + +struct db { + client: PickleDb, +}