diff --git a/Cargo.lock b/Cargo.lock index 2edc51a4ea..d094e4edc7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3521,7 +3521,9 @@ dependencies = [ "futures", "iceberg_test_utils", "itertools 0.13.0", + "log", "minijinja", + "miniz_oxide", "mockall", "moka", "murmur3", diff --git a/Cargo.toml b/Cargo.toml index c10c01d94a..349e33112d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -40,7 +40,7 @@ rust-version = "1.87" [workspace.dependencies] anyhow = "1.0.72" -apache-avro = { version = "0.20", features = ["zstandard"] } +apache-avro = { version = "0.20", features = ["zstandard", "snappy"] } array-init = "2" arrow-arith = "56.2" arrow-array = "56.2" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 895a5cf5e4..b1fec279b2 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -67,6 +67,8 @@ flate2 = { workspace = true } fnv = { workspace = true } futures = { workspace = true } itertools = { workspace = true } +log = { workspace = true } +miniz_oxide = "0.8" moka = { version = "0.12.10", features = ["future"] } murmur3 = { workspace = true } num-bigint = { workspace = true } diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index af297bebb5..36c8e3e6fd 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -194,8 +194,9 @@ mod tests { use crate::TableIdent; use crate::io::{FileIO, OutputFile}; use crate::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, Literal, ManifestEntry, - ManifestListWriter, ManifestStatus, ManifestWriterBuilder, Struct, TableMetadata, + CompressionSettings, DataContentType, DataFileBuilder, DataFileFormat, Literal, + ManifestEntry, ManifestListWriter, ManifestStatus, ManifestWriterBuilder, Struct, + TableMetadata, }; use crate::table::Table; @@ -275,6 +276,7 @@ mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), + CompressionSettings::default(), ) .build_v2_data(); writer @@ -307,6 +309,7 @@ mod tests { current_snapshot.snapshot_id(), current_snapshot.parent_snapshot_id(), current_snapshot.sequence_number(), + CompressionSettings::default(), ); manifest_list_write .add_manifests(vec![data_file_manifest].into_iter()) diff --git a/crates/iceberg/src/scan/mod.rs b/crates/iceberg/src/scan/mod.rs index 6884e00b9b..85009243b1 100644 --- a/crates/iceberg/src/scan/mod.rs +++ b/crates/iceberg/src/scan/mod.rs @@ -577,9 +577,9 @@ pub mod tests { use crate::io::{FileIO, OutputFile}; use crate::scan::FileScanTask; use crate::spec::{ - DataContentType, DataFileBuilder, DataFileFormat, Datum, Literal, ManifestEntry, - ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, PartitionSpec, - PrimitiveType, Schema, Struct, StructType, TableMetadata, Type, + CompressionSettings, DataContentType, DataFileBuilder, DataFileFormat, Datum, Literal, + ManifestEntry, ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, + PartitionSpec, PrimitiveType, Schema, Struct, StructType, TableMetadata, Type, }; use crate::table::Table; @@ -749,6 +749,7 @@ pub mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), + CompressionSettings::default(), ) .build_v2_data(); writer @@ -826,6 +827,7 @@ pub mod tests { current_snapshot.snapshot_id(), current_snapshot.parent_snapshot_id(), current_snapshot.sequence_number(), + CompressionSettings::default(), ); manifest_list_write .add_manifests(vec![data_file_manifest].into_iter()) @@ -961,6 +963,7 @@ pub mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), + CompressionSettings::default(), ) .build_v2_data(); @@ -1045,6 +1048,7 @@ pub mod tests { current_snapshot.snapshot_id(), current_snapshot.parent_snapshot_id(), current_snapshot.sequence_number(), + CompressionSettings::default(), ); manifest_list_write .add_manifests(vec![data_file_manifest].into_iter()) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs new file mode 100644 index 0000000000..5e0c863a83 --- /dev/null +++ b/crates/iceberg/src/spec/avro_util.rs @@ -0,0 +1,304 @@ +// 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. + +//! Utilities for working with Apache Avro in Iceberg. + +use apache_avro::Codec; +use log::warn; + +use crate::spec::TableProperties; + +/// Settings for compression codec and level. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct CompressionSettings { + /// The compression codec name (e.g., "gzip", "zstd", "snappy", "uncompressed") + pub codec: String, + /// The compression level (None uses codec-specific defaults) + pub level: Option, +} + +impl CompressionSettings { + /// Create a new CompressionSettings with the specified codec and level. + pub fn new(codec: String, level: Option) -> Self { + Self { codec, level } + } + + /// Convert to apache_avro::Codec using the codec_from_str helper function. + pub(crate) fn to_codec(&self) -> Codec { + codec_from_str(Some(&self.codec), self.level) + } +} + +impl Default for CompressionSettings { + fn default() -> Self { + Self { + codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + level: None, + } + } +} + +/// Convert codec name and level to apache_avro::Codec. +/// Returns Codec::Null for unknown or unsupported codecs. +/// +/// # Arguments +/// +/// * `codec` - The name of the compression codec (e.g., "gzip", "zstd", "snappy", "uncompressed") +/// * `level` - The compression level. For gzip/deflate: +/// - 0: NoCompression +/// - 1: BestSpeed +/// - 9: BestCompression +/// - 10: UberCompression +/// - 6: DefaultLevel (balanced speed/compression) +/// - Other values: DefaultLevel +/// For zstd, level is clamped to valid range (0-22). +/// When `None`, uses codec-specific defaults. +/// +/// # Supported Codecs +/// +/// - `gzip`: Uses Deflate compression with specified level +/// - `zstd`: Uses Zstandard compression (level clamped to valid zstd range 0-22) +/// - `snappy`: Uses Snappy compression (level parameter ignored) +/// - `uncompressed` or `None`: No compression +/// - Any other value: Defaults to no compression (Codec::Null) +pub(crate) fn codec_from_str(codec: Option<&str>, level: Option) -> Codec { + use apache_avro::{DeflateSettings, ZstandardSettings}; + + // Use case-insensitive comparison to match Java implementation + match codec.map(|s| s.to_lowercase()).as_deref() { + Some("gzip") => { + // Map compression level to miniz_oxide::deflate::CompressionLevel + // Reference: https://docs.rs/miniz_oxide/latest/miniz_oxide/deflate/enum.CompressionLevel.html + // Default level for gzip/deflate is 9 (BestCompression) to match Java + use miniz_oxide::deflate::CompressionLevel; + + let compression_level = match level.unwrap_or(9) { + 0 => CompressionLevel::NoCompression, + 1 => CompressionLevel::BestSpeed, + 9 => CompressionLevel::BestCompression, + 10 => CompressionLevel::UberCompression, + _ => CompressionLevel::DefaultLevel, + }; + + Codec::Deflate(DeflateSettings::new(compression_level)) + } + Some("zstd") => { + // Zstandard supports levels 0-22, clamp to valid range + // Default level for zstd is 1 to match Java + let zstd_level = level.unwrap_or(1).min(22); + Codec::Zstandard(ZstandardSettings::new(zstd_level)) + } + Some("snappy") => Codec::Snappy, + Some("uncompressed") | None => Codec::Null, + Some(unknown) => { + warn!( + "Unrecognized compression codec '{}', using no compression (Codec::Null)", + unknown + ); + Codec::Null + } + } +} + +#[cfg(test)] +mod tests { + use apache_avro::{DeflateSettings, ZstandardSettings}; + use miniz_oxide::deflate::CompressionLevel; + + use super::*; + + #[test] + fn test_codec_from_str_gzip() { + // Test with mixed case to verify case-insensitive matching + let codec = codec_from_str(Some("GZip"), Some(5)); + assert_eq!( + codec, + Codec::Deflate(DeflateSettings::new(CompressionLevel::DefaultLevel)) + ); + } + + #[test] + fn test_codec_from_str_snappy() { + let codec = codec_from_str(Some("snappy"), None); + assert_eq!(codec, Codec::Snappy); + } + + #[test] + fn test_codec_from_str_zstd() { + let codec = codec_from_str(Some("zstd"), Some(3)); + assert_eq!(codec, Codec::Zstandard(ZstandardSettings::new(3))); + } + + #[test] + fn test_codec_from_str_uncompressed() { + let codec = codec_from_str(Some("uncompressed"), None); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_null() { + let codec = codec_from_str(None, None); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_unknown() { + let codec = codec_from_str(Some("unknown"), Some(1)); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_gzip_default_level() { + // Test that None level defaults to 9 for gzip + let codec = codec_from_str(Some("gzip"), None); + assert_eq!( + codec, + Codec::Deflate(DeflateSettings::new(CompressionLevel::BestCompression)) + ); + } + + #[test] + fn test_codec_from_str_zstd_default_level() { + // Test that None level defaults to 1 for zstd + let codec = codec_from_str(Some("zstd"), None); + assert_eq!(codec, Codec::Zstandard(ZstandardSettings::new(1))); + } + + #[test] + fn test_codec_from_str_deflate_levels() { + use std::collections::HashMap; + + use apache_avro::types::Record; + use apache_avro::{Schema, Writer}; + + // Create a simple schema for testing + let schema = Schema::parse_str(r#"{"type": "record", "name": "test", "fields": [{"name": "field", "type": "string"}]}"#).unwrap(); + + // Create test data + let test_str = "test data that should compress differently at different levels. This is a longer string to ensure compression has something to work with. The quick brown fox jumps over the lazy dog. The quick brown fox jumps over the lazy dog."; + + // Test that different compression levels produce different output sizes + let mut sizes = HashMap::new(); + for level in [0, 1, 5, 9, 10] { + let codec = codec_from_str(Some("gzip"), Some(level)); + let mut writer = Writer::with_codec(&schema, Vec::new(), codec); + + let mut record = Record::new(&schema).unwrap(); + record.put("field", test_str); + writer.append(record).unwrap(); + + let encoded = writer.into_inner().unwrap(); + sizes.insert(level, encoded.len()); + } + + // Level 0 (NoCompression) should be largest + // Level 10 (UberCompression) should be smallest or equal to level 9 + assert!(sizes[&0] >= sizes[&1], "Level 0 should be >= level 1"); + assert!( + sizes[&1] >= sizes[&9] || sizes[&1] == sizes[&9], + "Level 1 should be >= level 9" + ); + assert!( + sizes[&9] >= sizes[&10] || sizes[&9] == sizes[&10], + "Level 9 should be >= level 10" + ); + } + + #[test] + fn test_codec_from_str_zstd_levels() { + use apache_avro::types::Record; + use apache_avro::{Schema, Writer}; + + // Create a simple schema for testing + let schema = Schema::parse_str(r#"{"type": "record", "name": "test", "fields": [{"name": "field", "type": "string"}]}"#).unwrap(); + let test_str = "test data that should compress differently at different levels. This is a longer string to ensure compression has something to work with. The quick brown fox jumps over the lazy dog. The quick brown fox jumps over the lazy dog."; + + // Test various levels by checking they produce valid codecs + for level in [0, 3, 15, 22] { + let codec = codec_from_str(Some("zstd"), Some(level)); + assert!(matches!(codec, Codec::Zstandard(_))); + + // Verify the codec actually works by compressing data + let mut writer = Writer::with_codec(&schema, Vec::new(), codec); + let mut record = Record::new(&schema).unwrap(); + record.put("field", test_str); + writer.append(record).unwrap(); + + let encoded = writer.into_inner().unwrap(); + assert!(!encoded.is_empty(), "Compression should produce output"); + } + + // Test clamping - higher than 22 should be clamped to 22 + let codec_100 = codec_from_str(Some("zstd"), Some(100)); + let codec_22 = codec_from_str(Some("zstd"), Some(22)); + + // Both should work and produce similar results + let mut writer_100 = Writer::with_codec(&schema, Vec::new(), codec_100); + let mut record_100 = Record::new(&schema).unwrap(); + record_100.put("field", test_str); + writer_100.append(record_100).unwrap(); + let encoded_100 = writer_100.into_inner().unwrap(); + + let mut writer_22 = Writer::with_codec(&schema, Vec::new(), codec_22); + let mut record_22 = Record::new(&schema).unwrap(); + record_22.put("field", test_str); + writer_22.append(record_22).unwrap(); + let encoded_22 = writer_22.into_inner().unwrap(); + + // Both should produce the same size since 100 is clamped to 22 + assert_eq!( + encoded_100.len(), + encoded_22.len(), + "Level 100 should be clamped to 22" + ); + } + + #[test] + fn test_compression_level_differences() { + use apache_avro::types::Record; + use apache_avro::{Schema, Writer}; + + // Create a schema and data that will compress well + let schema = Schema::parse_str(r#"{"type": "record", "name": "test", "fields": [{"name": "field", "type": "string"}]}"#).unwrap(); + + // Use highly compressible data + let test_str = "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"; + + // Test gzip level 0 (no compression) vs level 9 (best compression) + let codec_0 = codec_from_str(Some("gzip"), Some(0)); + let mut writer_0 = Writer::with_codec(&schema, Vec::new(), codec_0); + let mut record_0 = Record::new(&schema).unwrap(); + record_0.put("field", test_str); + writer_0.append(record_0).unwrap(); + let size_0 = writer_0.into_inner().unwrap().len(); + + let codec_9 = codec_from_str(Some("gzip"), Some(9)); + let mut writer_9 = Writer::with_codec(&schema, Vec::new(), codec_9); + let mut record_9 = Record::new(&schema).unwrap(); + record_9.put("field", test_str); + writer_9.append(record_9).unwrap(); + let size_9 = writer_9.into_inner().unwrap().len(); + + // Level 0 should produce larger output than level 9 for compressible data + assert!( + size_0 > size_9, + "NoCompression (level 0) should produce larger output than BestCompression (level 9): {} vs {}", + size_0, + size_9 + ); + } +} diff --git a/crates/iceberg/src/spec/manifest/mod.rs b/crates/iceberg/src/spec/manifest/mod.rs index 51219bfdb7..e3679433eb 100644 --- a/crates/iceberg/src/spec/manifest/mod.rs +++ b/crates/iceberg/src/spec/manifest/mod.rs @@ -166,7 +166,9 @@ mod tests { use super::*; use crate::io::FileIOBuilder; - use crate::spec::{Literal, NestedField, PrimitiveType, Struct, Transform, Type}; + use crate::spec::{ + CompressionSettings, Literal, NestedField, PrimitiveType, Struct, Transform, Type, + }; #[tokio::test] async fn test_parse_manifest_v2_unpartition() { @@ -272,6 +274,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -457,6 +460,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -554,6 +558,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v1(); for entry in &entries { @@ -663,6 +668,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v1(); for entry in &entries { @@ -771,6 +777,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -1050,6 +1057,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index ebb0590bcf..672e58407f 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -28,6 +28,7 @@ use super::{ }; use crate::error::Result; use crate::io::OutputFile; +use crate::spec::avro_util::CompressionSettings; use crate::spec::manifest::_serde::{ManifestEntryV1, ManifestEntryV2}; use crate::spec::manifest::{manifest_schema_v1, manifest_schema_v2}; use crate::spec::{ @@ -43,6 +44,7 @@ pub struct ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, + compression: CompressionSettings, } impl ManifestWriterBuilder { @@ -53,6 +55,7 @@ impl ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, + compression: CompressionSettings, ) -> Self { Self { output, @@ -60,6 +63,7 @@ impl ManifestWriterBuilder { key_metadata, schema, partition_spec, + compression, } } @@ -78,6 +82,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression, ) } @@ -96,6 +101,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression, ) } @@ -114,6 +120,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression, ) } @@ -134,6 +141,7 @@ impl ManifestWriterBuilder { // First row id is assigned by the [`ManifestListWriter`] when the manifest // is added to the list. None, + self.compression, ) } @@ -152,6 +160,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression, ) } } @@ -177,6 +186,8 @@ pub struct ManifestWriter { manifest_entries: Vec, metadata: ManifestMetadata, + + compression: CompressionSettings, } impl ManifestWriter { @@ -187,6 +198,7 @@ impl ManifestWriter { key_metadata: Option>, metadata: ManifestMetadata, first_row_id: Option, + compression: CompressionSettings, ) -> Self { Self { output, @@ -202,6 +214,7 @@ impl ManifestWriter { key_metadata, manifest_entries: Vec::new(), metadata, + compression, } } @@ -410,7 +423,10 @@ impl ManifestWriter { // Manifest schema did not change between V2 and V3 FormatVersion::V2 | FormatVersion::V3 => manifest_schema_v2(&partition_type)?, }; - let mut avro_writer = AvroWriter::new(&avro_schema, Vec::new()); + + let codec = self.compression.to_codec(); + + let mut avro_writer = AvroWriter::with_codec(&avro_schema, Vec::new(), codec); avro_writer.add_user_metadata( "schema".to_string(), to_vec(table_schema).map_err(|err| { @@ -689,6 +705,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); writer.add_entry(entries[0].clone()).unwrap(); @@ -708,4 +725,132 @@ mod tests { entries[0].file_sequence_number = None; assert_eq!(actual_manifest, Manifest::new(metadata, entries)); } + + #[tokio::test] + async fn test_manifest_writer_with_compression() { + use std::fs; + + use tempfile::TempDir; + + use crate::io::FileIOBuilder; + use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, ManifestContentType, ManifestEntry, + ManifestMetadata, ManifestStatus, NestedField, PartitionSpec, PrimitiveType, Schema, + }; + + let metadata = { + let schema = Schema::builder() + .with_fields(vec![Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Int), + ))]) + .build() + .unwrap(); + + ManifestMetadata { + schema_id: 0, + schema: Arc::new(schema), + partition_spec: PartitionSpec::unpartition_spec(), + format_version: FormatVersion::V2, + content: ManifestContentType::Data, + } + }; + + // Write uncompressed manifest with multiple entries to make compression effective + let tmp_dir = TempDir::new().unwrap(); + let uncompressed_path = tmp_dir.path().join("uncompressed_manifest.avro"); + let io = FileIOBuilder::new_fs_io().build().unwrap(); + let output_file = io.new_output(uncompressed_path.to_str().unwrap()).unwrap(); + let uncompressed_settings = CompressionSettings::new("uncompressed".to_string(), None); + let mut writer = ManifestWriterBuilder::new( + output_file, + Some(1), + None, + metadata.schema.clone(), + metadata.partition_spec.clone(), + uncompressed_settings, + ) + .build_v2_data(); + // Add multiple entries with long paths to create compressible data + for i in 0..1000 { + let data_file = DataFileBuilder::default() + .content(DataContentType::Data) + .file_path(format!( + "/very/long/path/to/data/directory/with/many/subdirectories/file_{}.parquet", + i + )) + .file_format(DataFileFormat::Parquet) + .partition(Struct::empty()) + .file_size_in_bytes(100000 + i) + .record_count(1000 + i) + .build() + .unwrap(); + + let entry = ManifestEntry::builder() + .status(ManifestStatus::Added) + .snapshot_id(1) + .sequence_number(1) + .file_sequence_number(1) + .data_file(data_file) + .build(); + writer.add_entry(entry).unwrap(); + } + writer.write_manifest_file().await.unwrap(); + let uncompressed_size = fs::metadata(&uncompressed_path).unwrap().len(); + + // Write compressed manifest with gzip + let compressed_path = tmp_dir.path().join("compressed_manifest.avro"); + let output_file = io.new_output(compressed_path.to_str().unwrap()).unwrap(); + let compression = CompressionSettings::new("gzip".to_string(), Some(9)); + let mut writer = ManifestWriterBuilder::new( + output_file, + Some(1), + None, + metadata.schema.clone(), + metadata.partition_spec.clone(), + compression, + ) + .build_v2_data(); + // Add the same entries with long paths as the uncompressed version + for i in 0..1000 { + let data_file = DataFileBuilder::default() + .content(DataContentType::Data) + .file_path(format!( + "/very/long/path/to/data/directory/with/many/subdirectories/file_{}.parquet", + i + )) + .file_format(DataFileFormat::Parquet) + .partition(Struct::empty()) + .file_size_in_bytes(100000 + i) + .record_count(1000 + i) + .build() + .unwrap(); + + let entry = ManifestEntry::builder() + .status(ManifestStatus::Added) + .snapshot_id(1) + .sequence_number(1) + .file_sequence_number(1) + .data_file(data_file) + .build(); + writer.add_entry(entry).unwrap(); + } + writer.write_manifest_file().await.unwrap(); + let compressed_size = fs::metadata(&compressed_path).unwrap().len(); + + // Verify compression is actually working + assert!( + compressed_size < uncompressed_size, + "Compressed size ({}) should be less than uncompressed size ({})", + compressed_size, + uncompressed_size + ); + + // Verify the compressed file can be read back correctly + let compressed_bytes = fs::read(&compressed_path).unwrap(); + let manifest = Manifest::parse_avro(&compressed_bytes).unwrap(); + assert_eq!(manifest.metadata.format_version, FormatVersion::V2); + assert_eq!(manifest.entries.len(), 1000); + } } diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 5e97e5466e..a5be5c2198 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -31,6 +31,7 @@ use self::_serde::{ManifestFileV1, ManifestFileV2}; use super::{FormatVersion, Manifest}; use crate::error::Result; use crate::io::{FileIO, OutputFile}; +use crate::spec::avro_util::CompressionSettings; use crate::spec::manifest_list::_const_schema::MANIFEST_LIST_AVRO_SCHEMA_V3; use crate::spec::manifest_list::_serde::ManifestFileV3; use crate::{Error, ErrorKind}; @@ -117,7 +118,12 @@ impl ManifestListWriter { } /// Construct a v1 [`ManifestListWriter`] that writes to a provided [`OutputFile`]. - pub fn v1(output_file: OutputFile, snapshot_id: i64, parent_snapshot_id: Option) -> Self { + pub fn v1( + output_file: OutputFile, + snapshot_id: i64, + parent_snapshot_id: Option, + compression_settings: CompressionSettings, + ) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("format-version".to_string(), "1".to_string()), @@ -135,6 +141,7 @@ impl ManifestListWriter { 0, snapshot_id, None, + compression_settings, ) } @@ -144,6 +151,7 @@ impl ManifestListWriter { snapshot_id: i64, parent_snapshot_id: Option, sequence_number: i64, + compression_settings: CompressionSettings, ) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), @@ -163,6 +171,7 @@ impl ManifestListWriter { sequence_number, snapshot_id, None, + compression_settings, ) } @@ -173,6 +182,7 @@ impl ManifestListWriter { parent_snapshot_id: Option, sequence_number: i64, first_row_id: Option, // Always None for delete manifests + compression_settings: CompressionSettings, ) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), @@ -198,6 +208,7 @@ impl ManifestListWriter { sequence_number, snapshot_id, first_row_id, + compression_settings, ) } @@ -208,13 +219,18 @@ impl ManifestListWriter { sequence_number: i64, snapshot_id: i64, first_row_id: Option, + compression: CompressionSettings, ) -> Self { let avro_schema = match format_version { FormatVersion::V1 => &MANIFEST_LIST_AVRO_SCHEMA_V1, FormatVersion::V2 => &MANIFEST_LIST_AVRO_SCHEMA_V2, FormatVersion::V3 => &MANIFEST_LIST_AVRO_SCHEMA_V3, }; - let mut avro_writer = Writer::new(avro_schema, Vec::new()); + + // Use CompressionSettings to determine compression codec + let codec = compression.to_codec(); + + let mut avro_writer = Writer::with_codec(avro_schema, Vec::new(), codec); for (key, value) in metadata { avro_writer .add_user_metadata(key, value) @@ -1368,8 +1384,8 @@ mod test { use crate::io::FileIOBuilder; use crate::spec::manifest_list::_serde::{ManifestListV1, ManifestListV3}; use crate::spec::{ - Datum, FieldSummary, ManifestContentType, ManifestFile, ManifestList, ManifestListWriter, - UNASSIGNED_SEQUENCE_NUMBER, + CompressionSettings, Datum, FieldSummary, ManifestContentType, ManifestFile, ManifestList, + ManifestListWriter, UNASSIGNED_SEQUENCE_NUMBER, }; #[tokio::test] @@ -1407,6 +1423,7 @@ mod test { file_io.new_output(full_path.clone()).unwrap(), 1646658105718557341, Some(1646658105718557341), + CompressionSettings::default(), ); writer @@ -1480,6 +1497,7 @@ mod test { 1646658105718557341, Some(1646658105718557341), 1, + CompressionSettings::default(), ); writer @@ -1554,6 +1572,7 @@ mod test { Some(377075049360453639), 1, Some(10), + CompressionSettings::default(), ); writer @@ -1690,7 +1709,12 @@ mod test { 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, Some(0)); + let mut writer = ManifestListWriter::v1( + output_file, + 1646658105718557341, + Some(0), + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1737,7 +1761,13 @@ mod test { 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, Some(0), seq_num); + let mut writer = ManifestListWriter::v2( + output_file, + snapshot_id, + Some(0), + seq_num, + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1785,8 +1815,14 @@ mod test { let io = FileIOBuilder::new_fs_io().build().unwrap(); let output_file = io.new_output(path.to_str().unwrap()).unwrap(); - let mut writer = - ManifestListWriter::v3(output_file, snapshot_id, Some(0), seq_num, Some(10)); + let mut writer = ManifestListWriter::v3( + output_file, + snapshot_id, + Some(0), + seq_num, + Some(10), + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1833,7 +1869,12 @@ mod test { 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, Some(0)); + let mut writer = ManifestListWriter::v1( + output_file, + 1646658105718557341, + Some(0), + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1878,7 +1919,12 @@ mod test { 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, Some(0)); + let mut writer = ManifestListWriter::v1( + output_file, + 1646658105718557341, + Some(0), + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1925,7 +1971,13 @@ mod test { 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, Some(0), seq_num); + let mut writer = ManifestListWriter::v2( + output_file, + snapshot_id, + Some(0), + seq_num, + CompressionSettings::default(), + ); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -2044,4 +2096,96 @@ mod test { assert_eq!(v2_manifest.partitions, None); assert_eq!(v2_manifest.key_metadata, None); } + + #[tokio::test] + async fn test_manifest_list_writer_with_compression() { + use std::fs; + + use tempfile::TempDir; + + use crate::io::FileIOBuilder; + + // Create multiple manifest entries to make compression effective + let mut entries = Vec::new(); + for i in 0..100 { + entries.push(ManifestFile { + manifest_path: format!("/test/manifest{}.avro", i), + manifest_length: 1000 + i, + partition_spec_id: 0, + content: ManifestContentType::Data, + sequence_number: 1, + min_sequence_number: 1, + added_snapshot_id: 1646658105718557341, + added_files_count: Some(10), + existing_files_count: Some(5), + deleted_files_count: Some(2), + added_rows_count: Some(100), + existing_rows_count: Some(50), + deleted_rows_count: Some(20), + partitions: None, + key_metadata: None, + first_row_id: None, + }); + } + let manifest_list = ManifestList { entries }; + + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + let tmp_dir = TempDir::new().unwrap(); + + // Write uncompressed manifest list + let uncompressed_path = tmp_dir + .path() + .join("uncompressed_manifest_list.avro") + .to_str() + .unwrap() + .to_string(); + let mut writer = ManifestListWriter::v2( + file_io.new_output(&uncompressed_path).unwrap(), + 1646658105718557341, + Some(0), + 1, + CompressionSettings::new("uncompressed".to_string(), None), + ); + writer + .add_manifests(manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + let uncompressed_size = fs::metadata(&uncompressed_path).unwrap().len(); + + // Write compressed manifest list with gzip + let compressed_path = tmp_dir + .path() + .join("compressed_manifest_list.avro") + .to_str() + .unwrap() + .to_string(); + let compression = CompressionSettings::new("gzip".to_string(), Some(9)); + let mut writer = ManifestListWriter::v2( + file_io.new_output(&compressed_path).unwrap(), + 1646658105718557341, + Some(0), + 1, + compression, + ); + writer + .add_manifests(manifest_list.entries.clone().into_iter()) + .unwrap(); + writer.close().await.unwrap(); + let compressed_size = fs::metadata(&compressed_path).unwrap().len(); + + // Verify compression is actually working + assert!( + compressed_size < uncompressed_size, + "Compressed size ({}) should be less than uncompressed size ({})", + compressed_size, + uncompressed_size + ); + + // Verify the compressed file can be read back correctly + let compressed_bytes = fs::read(&compressed_path).unwrap(); + let parsed_manifest_list = + ManifestList::parse_with_version(&compressed_bytes, crate::spec::FormatVersion::V2) + .unwrap(); + assert_eq!(manifest_list, parsed_manifest_list); + } } diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs index 44b35e5a6b..bb8a50d828 100644 --- a/crates/iceberg/src/spec/mod.rs +++ b/crates/iceberg/src/spec/mod.rs @@ -17,6 +17,7 @@ //! Spec for Iceberg. +mod avro_util; mod datatypes; mod encrypted_key; mod manifest; @@ -37,6 +38,7 @@ mod view_metadata; mod view_metadata_builder; mod view_version; +pub use avro_util::CompressionSettings; pub use datatypes::*; pub use encrypted_key::*; pub use manifest::*; diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index 4975456010..fb5d990122 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -16,21 +16,28 @@ // under the License. use std::collections::HashMap; +use std::fmt::Display; +use std::str::FromStr; + +use crate::error::{Error, ErrorKind}; // Helper function to parse a property from a HashMap // If the property is not found, use the default value -fn parse_property( +fn parse_property( properties: &HashMap, key: &str, default: T, -) -> Result +) -> crate::error::Result where - ::Err: std::fmt::Display, + ::Err: Display, { properties.get(key).map_or(Ok(default), |value| { - value - .parse::() - .map_err(|e| anyhow::anyhow!("Invalid value for {key}: {e}")) + value.parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Invalid value for {key}: {e}"), + ) + }) }) } @@ -49,6 +56,10 @@ pub struct TableProperties { pub write_format_default: String, /// The target file size for files. pub write_target_file_size_bytes: usize, + /// Compression codec for Avro files (manifests, manifest lists) + pub avro_compression_codec: String, + /// Compression level for Avro files (None uses codec-specific defaults) + pub avro_compression_level: Option, } impl TableProperties { @@ -137,11 +148,19 @@ impl TableProperties { pub const PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES: &str = "write.target-file-size-bytes"; /// Default target file size pub const PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT: usize = 512 * 1024 * 1024; // 512 MB + + /// Compression codec for Avro files (manifests, manifest lists) + pub const PROPERTY_AVRO_COMPRESSION_CODEC: &str = "write.avro.compression-codec"; + /// Default Avro compression codec - gzip + pub const PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT: &str = "gzip"; + + /// Compression level for Avro files + pub const PROPERTY_AVRO_COMPRESSION_LEVEL: &str = "write.avro.compression-level"; } impl TryFrom<&HashMap> for TableProperties { // parse by entry key or use default value - type Error = anyhow::Error; + type Error = crate::Error; fn try_from(props: &HashMap) -> Result { Ok(TableProperties { @@ -175,6 +194,19 @@ impl TryFrom<&HashMap> for TableProperties { TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, )?, + avro_compression_codec: parse_property( + props, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + )?, + avro_compression_level: { + let level = parse_property( + props, + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL, + 255u8, + )?; + if level == 255 { None } else { Some(level) } + }, }) } } @@ -207,6 +239,29 @@ mod tests { table_properties.write_target_file_size_bytes, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT ); + // Test compression defaults + assert_eq!( + table_properties.avro_compression_codec, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string() + ); + assert_eq!(table_properties.avro_compression_level, None); + } + + #[test] + fn test_table_properties_compression() { + let props = HashMap::from([ + ( + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC.to_string(), + "zstd".to_string(), + ), + ( + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL.to_string(), + "3".to_string(), + ), + ]); + let table_properties = TableProperties::try_from(&props).unwrap(); + assert_eq!(table_properties.avro_compression_codec, "zstd"); + assert_eq!(table_properties.avro_compression_level, Some(3)); } #[test] @@ -281,4 +336,34 @@ mod tests { "Invalid value for write.target-file-size-bytes: invalid digit found in string" )); } + + #[test] + fn test_table_properties_optional_compression_level() { + // Test that compression level is None when not specified + let props = HashMap::new(); + let table_properties = TableProperties::try_from(&props).unwrap(); + assert_eq!(table_properties.avro_compression_level, None); + + // Test that compression level is Some(value) when specified + let props = HashMap::from([( + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL.to_string(), + "5".to_string(), + )]); + let table_properties = TableProperties::try_from(&props).unwrap(); + assert_eq!(table_properties.avro_compression_level, Some(5)); + + // Test that invalid compression level returns error + let props = HashMap::from([( + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL.to_string(), + "invalid".to_string(), + )]); + let result = TableProperties::try_from(&props); + assert!(result.is_err()); + assert!( + result + .unwrap_err() + .to_string() + .contains("Invalid value for write.avro.compression-level") + ); + } } diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 4f85962ff1..5a3414ac02 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -23,10 +23,10 @@ use uuid::Uuid; use crate::error::Result; use crate::spec::{ - DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, ManifestEntry, - ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, Operation, Snapshot, - SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, StructType, Summary, - TableProperties, update_snapshot_summaries, + CompressionSettings, DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, + ManifestEntry, ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, + Operation, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, + StructType, Summary, TableProperties, update_snapshot_summaries, }; use crate::table::Table; use crate::transaction::ActionCommit; @@ -195,6 +195,21 @@ impl<'a> SnapshotProducer<'a> { DataFileFormat::Avro ); let output_file = self.table.file_io().new_output(new_manifest_path)?; + + // Get compression settings from table properties + let table_props = + TableProperties::try_from(self.table.metadata().properties()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + "Failed to parse table properties for compression settings", + ) + .with_source(e) + })?; + let compression = CompressionSettings::new( + table_props.avro_compression_codec, + table_props.avro_compression_level, + ); + let builder = ManifestWriterBuilder::new( output_file, Some(self.snapshot_id), @@ -205,7 +220,9 @@ impl<'a> SnapshotProducer<'a> { .default_partition_spec() .as_ref() .clone(), + compression, ); + match self.table.metadata().format_version() { FormatVersion::V1 => Ok(builder.build_v1()), FormatVersion::V2 => match content { @@ -386,6 +403,21 @@ impl<'a> SnapshotProducer<'a> { let manifest_list_path = self.generate_manifest_list_file_path(0); let next_seq_num = self.table.metadata().next_sequence_number(); let first_row_id = self.table.metadata().next_row_id(); + + // Get compression settings from table properties + let table_props = + TableProperties::try_from(self.table.metadata().properties()).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + "Failed to parse table properties for compression settings", + ) + .with_source(e) + })?; + let compression = CompressionSettings::new( + table_props.avro_compression_codec, + table_props.avro_compression_level, + ); + let mut manifest_list_writer = match self.table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( self.table @@ -393,6 +425,7 @@ impl<'a> SnapshotProducer<'a> { .new_output(manifest_list_path.clone())?, self.snapshot_id, self.table.metadata().current_snapshot_id(), + compression, ), FormatVersion::V2 => ManifestListWriter::v2( self.table @@ -401,6 +434,7 @@ impl<'a> SnapshotProducer<'a> { self.snapshot_id, self.table.metadata().current_snapshot_id(), next_seq_num, + compression, ), FormatVersion::V3 => ManifestListWriter::v3( self.table @@ -410,6 +444,7 @@ impl<'a> SnapshotProducer<'a> { self.table.metadata().current_snapshot_id(), next_seq_num, Some(first_row_id), + compression, ), };