From 665b970af76d4612800c8eba55f9cc531678b157 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Wed, 12 Nov 2025 23:00:56 +0000 Subject: [PATCH 01/25] feat!: Support compression codecs for JSON metadata and Avro Previously these properties where not honored on tabel properties. - Adds table properties for these values. - Plumbs them through for writers. --- Cargo.lock | 2 + crates/iceberg/Cargo.toml | 2 + crates/iceberg/src/spec/avro_util.rs | 230 ++++++++++++++++++++ crates/iceberg/src/spec/manifest/writer.rs | 37 +++- crates/iceberg/src/spec/manifest_list.rs | 49 ++++- crates/iceberg/src/spec/mod.rs | 1 + crates/iceberg/src/spec/table_metadata.rs | 115 +++++++++- crates/iceberg/src/spec/table_properties.rs | 71 ++++++ crates/iceberg/src/transaction/snapshot.rs | 33 ++- 9 files changed, 533 insertions(+), 7 deletions(-) create mode 100644 crates/iceberg/src/spec/avro_util.rs 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/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/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs new file mode 100644 index 0000000000..055b304f47 --- /dev/null +++ b/crates/iceberg/src/spec/avro_util.rs @@ -0,0 +1,230 @@ +// 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; + +/// 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", "deflate", "none") +/// * `level` - The compression level. For deflate/gzip: +/// - 0: NoCompression +/// - 1: BestSpeed +/// - 9: BestCompression +/// - 10: UberCompression +/// - Other values: DefaultLevel (6) +/// +/// # Supported Codecs +/// +/// - `gzip` or `deflate`: Uses Deflate compression with specified level +/// - `zstd`: Uses Zstandard compression (level clamped to valid zstd range) +/// - `none` or `None`: No compression +/// - Any other value: Defaults to no compression (Codec::Null) +/// +/// # Compression Levels +/// +/// The compression level mapping is based on miniz_oxide's CompressionLevel enum: +/// - Level 0: No compression +/// - Level 1: Best speed (fastest) +/// - Level 9: Best compression (slower, better compression) +/// - Level 10: Uber compression (slowest, best compression) +/// - Other: Default level (balanced speed/compression) +pub(crate) fn codec_from_str(codec: Option<&str>, level: u8) -> Codec { + use apache_avro::{DeflateSettings, ZstandardSettings}; + + match codec { + Some("gzip") | Some("deflate") => { + // Map compression level to miniz_oxide::deflate::CompressionLevel + // Reference: https://docs.rs/miniz_oxide/latest/miniz_oxide/deflate/enum.CompressionLevel.html + use miniz_oxide::deflate::CompressionLevel; + + let compression_level = match level { + 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 + let zstd_level = level.min(22); + Codec::Zstandard(ZstandardSettings::new(zstd_level)) + } + Some("none") | None => Codec::Null, + Some(unknown) => { + warn!( + "Unrecognized compression codec '{}', using no compression (Codec::Null)", + unknown + ); + Codec::Null + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_codec_from_str_gzip() { + let codec = codec_from_str(Some("gzip"), 5); + assert!(matches!(codec, Codec::Deflate(_))); + } + + #[test] + fn test_codec_from_str_deflate() { + let codec = codec_from_str(Some("deflate"), 9); + assert!(matches!(codec, Codec::Deflate(_))); + } + + #[test] + fn test_codec_from_str_zstd() { + let codec = codec_from_str(Some("zstd"), 3); + assert!(matches!(codec, Codec::Zstandard(_))); + } + + #[test] + fn test_codec_from_str_none() { + let codec = codec_from_str(Some("none"), 0); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_null() { + let codec = codec_from_str(None, 0); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_unknown() { + let codec = codec_from_str(Some("unknown"), 1); + assert!(matches!(codec, Codec::Null)); + } + + #[test] + fn test_codec_from_str_deflate_levels() { + use apache_avro::{Writer, Schema, types::Record}; + use std::collections::HashMap; + + // 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"), 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::{Writer, Schema, types::Record}; + + // 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"), 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.len() > 0, "Compression should produce output"); + } + + // Test clamping - higher than 22 should be clamped to 22 + let codec_100 = codec_from_str(Some("zstd"), 100); + let codec_22 = codec_from_str(Some("zstd"), 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::{Writer, Schema, types::Record}; + + // 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"), 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"), 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/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index ebb0590bcf..aa727c1388 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::codec_from_str; use crate::spec::manifest::_serde::{ManifestEntryV1, ManifestEntryV2}; use crate::spec::manifest::{manifest_schema_v1, manifest_schema_v2}; use crate::spec::{ @@ -43,6 +44,8 @@ pub struct ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, + compression_codec: String, + compression_level: u8, } impl ManifestWriterBuilder { @@ -54,15 +57,26 @@ impl ManifestWriterBuilder { schema: SchemaRef, partition_spec: PartitionSpec, ) -> Self { + use crate::spec::TableProperties; + Self { output, snapshot_id, key_metadata, schema, partition_spec, + compression_codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + compression_level: TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, } } + /// Set compression codec and level for the manifest file. + pub fn with_compression(mut self, codec: String, level: u8) -> Self { + self.compression_codec = codec; + self.compression_level = level; + self + } + /// Build a [`ManifestWriter`] for format version 1. pub fn build_v1(self) -> ManifestWriter { let metadata = ManifestMetadata::builder() @@ -78,6 +92,8 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression_codec, + self.compression_level, ) } @@ -96,6 +112,8 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression_codec, + self.compression_level, ) } @@ -114,6 +132,8 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression_codec, + self.compression_level, ) } @@ -134,6 +154,8 @@ impl ManifestWriterBuilder { // First row id is assigned by the [`ManifestListWriter`] when the manifest // is added to the list. None, + self.compression_codec, + self.compression_level, ) } @@ -152,6 +174,8 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, + self.compression_codec, + self.compression_level, ) } } @@ -177,6 +201,9 @@ pub struct ManifestWriter { manifest_entries: Vec, metadata: ManifestMetadata, + + compression_codec: String, + compression_level: u8, } impl ManifestWriter { @@ -187,6 +214,8 @@ impl ManifestWriter { key_metadata: Option>, metadata: ManifestMetadata, first_row_id: Option, + compression_codec: String, + compression_level: u8, ) -> Self { Self { output, @@ -202,6 +231,8 @@ impl ManifestWriter { key_metadata, manifest_entries: Vec::new(), metadata, + compression_codec, + compression_level, } } @@ -410,7 +441,11 @@ 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()); + + // Determine compression codec using helper function + let codec = codec_from_str(Some(self.compression_codec.as_str()), self.compression_level); + + 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| { diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 5e97e5466e..fd41f33e12 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::codec_from_str; use crate::spec::manifest_list::_const_schema::MANIFEST_LIST_AVRO_SCHEMA_V3; use crate::spec::manifest_list::_serde::ManifestFileV3; use crate::{Error, ErrorKind}; @@ -98,6 +99,8 @@ pub struct ManifestListWriter { sequence_number: i64, snapshot_id: i64, next_row_id: Option, + compression_codec: String, + compression_level: u8, } impl std::fmt::Debug for ManifestListWriter { @@ -118,6 +121,8 @@ 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 { + use crate::spec::TableProperties; + let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("format-version".to_string(), "1".to_string()), @@ -135,6 +140,8 @@ impl ManifestListWriter { 0, snapshot_id, None, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, ) } @@ -145,6 +152,8 @@ impl ManifestListWriter { parent_snapshot_id: Option, sequence_number: i64, ) -> Self { + use crate::spec::TableProperties; + let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("sequence-number".to_string(), sequence_number.to_string()), @@ -163,6 +172,8 @@ impl ManifestListWriter { sequence_number, snapshot_id, None, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, ) } @@ -174,6 +185,8 @@ impl ManifestListWriter { sequence_number: i64, first_row_id: Option, // Always None for delete manifests ) -> Self { + use crate::spec::TableProperties; + let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("sequence-number".to_string(), sequence_number.to_string()), @@ -198,9 +211,35 @@ impl ManifestListWriter { sequence_number, snapshot_id, first_row_id, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, ) } + /// Set compression codec and level for the manifest list file. + pub fn with_compression(mut self, codec: String, level: u8) -> Self { + self.compression_codec = codec.clone(); + self.compression_level = level; + + // Recreate the avro_writer with the new codec + let avro_schema = match self.format_version { + FormatVersion::V1 => &MANIFEST_LIST_AVRO_SCHEMA_V1, + FormatVersion::V2 => &MANIFEST_LIST_AVRO_SCHEMA_V2, + FormatVersion::V3 => &MANIFEST_LIST_AVRO_SCHEMA_V3, + }; + + // Use helper function to get codec + let compression = codec_from_str(Some(codec.as_str()), level); + + let new_writer = Writer::with_codec(avro_schema, Vec::new(), compression); + + // Copy over existing metadata from the old writer + // Unfortunately, we can't extract metadata from the old writer, + // so we'll need to handle this differently + self.avro_writer = new_writer; + self + } + fn new( format_version: FormatVersion, output_file: OutputFile, @@ -208,13 +247,19 @@ impl ManifestListWriter { sequence_number: i64, snapshot_id: i64, first_row_id: Option, + compression_codec: String, + compression_level: u8, ) -> 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 helper function to determine compression codec + let codec = codec_from_str(Some(compression_codec.as_str()), compression_level); + + let mut avro_writer = Writer::with_codec(avro_schema, Vec::new(), codec); for (key, value) in metadata { avro_writer .add_user_metadata(key, value) @@ -227,6 +272,8 @@ impl ManifestListWriter { sequence_number, snapshot_id, next_row_id: first_row_id, + compression_codec, + compression_level, } } diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs index 44b35e5a6b..cfe9132193 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; diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 06b32cc847..24c0fb99a5 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -461,9 +461,57 @@ impl TableMetadata { file_io: &FileIO, metadata_location: impl AsRef, ) -> Result<()> { + use std::io::Write as _; + + use flate2::write::GzEncoder; + + let json_data = serde_json::to_vec(self)?; + + // Check if compression is enabled via table properties + let codec = self + .properties + .get(crate::spec::table_properties::TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .map(|s| s.as_str()) + .unwrap_or(crate::spec::table_properties::TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT); + + let (data_to_write, actual_location) = match codec { + "gzip" => { + let mut encoder = GzEncoder::new(Vec::new(), flate2::Compression::default()); + encoder.write_all(&json_data).map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + "Failed to compress metadata with gzip", + ) + .with_source(e) + })?; + let compressed_data = encoder.finish().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Failed to finish gzip compression") + .with_source(e) + })?; + + // Modify filename to add .gz before .metadata.json + let location = metadata_location.as_ref(); + let new_location = if location.ends_with(".metadata.json") { + location.replace(".metadata.json", ".gz.metadata.json") + } else { + // If it doesn't end with expected pattern, just append .gz + format!("{}.gz", location) + }; + + (compressed_data, new_location) + } + "none" | "" => (json_data, metadata_location.as_ref().to_string()), + other => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!("Unsupported metadata compression codec: {}", other), + )); + } + }; + file_io - .new_output(metadata_location)? - .write(serde_json::to_vec(self)?.into()) + .new_output(actual_location)? + .write(data_to_write.into()) .await } @@ -1556,7 +1604,7 @@ mod tests { BlobMetadata, EncryptedKey, INITIAL_ROW_ID, Literal, NestedField, NullOrder, Operation, PartitionSpec, PartitionStatisticsFile, PrimitiveLiteral, PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, StatisticsFile, - Summary, Transform, Type, UnboundPartitionField, + Summary, TableProperties, Transform, Type, UnboundPartitionField, }; fn check_table_metadata_serde(json: &str, expected_type: TableMetadata) { @@ -3584,6 +3632,67 @@ mod tests { assert!(result.is_err()); } + #[tokio::test] + async fn test_table_metadata_write_with_gzip_compression() { + let temp_dir = TempDir::new().unwrap(); + let temp_path = temp_dir.path().to_str().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + + // Get a test metadata and add gzip compression property + let original_metadata: TableMetadata = get_test_table_metadata("TableMetadataV2Valid.json"); + + // Modify properties to enable gzip compression + let mut props = original_metadata.properties.clone(); + props.insert( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "gzip".to_string(), + ); + // Use builder to create new metadata with updated properties + let compressed_metadata = + TableMetadataBuilder::new_from_metadata(original_metadata.clone(), None) + .assign_uuid(original_metadata.table_uuid) + .set_properties(props.clone()) + .unwrap() + .build() + .unwrap() + .metadata; + + // Write the metadata with compression - note the location will be modified to add .gz + let metadata_location = format!("{temp_path}/00001-test.metadata.json"); + compressed_metadata + .write_to(&file_io, &metadata_location) + .await + .unwrap(); + + // The actual file should be written with .gz.metadata.json extension + let expected_compressed_location = format!("{temp_path}/00001-test.gz.metadata.json"); + + // Verify the compressed file exists + assert!(std::path::Path::new(&expected_compressed_location).exists()); + + // Read the raw file and check it's gzip compressed + let raw_content = std::fs::read(&expected_compressed_location).unwrap(); + assert!(raw_content.len() > 2); + assert_eq!(raw_content[0], 0x1F); // gzip magic number + assert_eq!(raw_content[1], 0x8B); // gzip magic number + + // Read the metadata back using the compressed location + let read_metadata = TableMetadata::read_from(&file_io, &expected_compressed_location) + .await + .unwrap(); + + // Verify the properties include the compression codec + assert_eq!( + read_metadata + .properties + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC), + Some(&"gzip".to_string()) + ); + + // Verify the complete round-trip: read metadata should match what we wrote + assert_eq!(read_metadata, compressed_metadata); + } + #[test] fn test_partition_name_exists() { let schema = Schema::builder() diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index 4975456010..10908362ca 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -49,6 +49,12 @@ pub struct TableProperties { pub write_format_default: String, /// The target file size for files. pub write_target_file_size_bytes: usize, + /// Compression codec for metadata files (JSON) + pub metadata_compression_codec: String, + /// Compression codec for Avro files (manifests, manifest lists) + pub avro_compression_codec: String, + /// Compression level for Avro files + pub avro_compression_level: u8, } impl TableProperties { @@ -137,6 +143,21 @@ 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 metadata files (JSON) + pub const PROPERTY_METADATA_COMPRESSION_CODEC: &str = "write.metadata.compression-codec"; + /// Default metadata compression codec - none + pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "none"; + + /// 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"; + /// Default Avro compression level (9 = BestCompression) + pub const PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT: u8 = 9; } impl TryFrom<&HashMap> for TableProperties { @@ -175,6 +196,21 @@ impl TryFrom<&HashMap> for TableProperties { TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, )?, + metadata_compression_codec: parse_property( + props, + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC, + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string(), + )?, + avro_compression_codec: parse_property( + props, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + )?, + avro_compression_level: parse_property( + props, + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL, + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + )?, }) } } @@ -207,6 +243,41 @@ mod tests { table_properties.write_target_file_size_bytes, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT ); + // Test compression defaults + assert_eq!( + table_properties.metadata_compression_codec, + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string() + ); + assert_eq!( + table_properties.avro_compression_codec, + TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string() + ); + assert_eq!( + table_properties.avro_compression_level, + TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT + ); + } + + #[test] + fn test_table_properties_compression() { + let props = HashMap::from([ + ( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "gzip".to_string(), + ), + ( + 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.metadata_compression_codec, "gzip"); + assert_eq!(table_properties.avro_compression_codec, "zstd"); + assert_eq!(table_properties.avro_compression_level, 3); } #[test] diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 4f85962ff1..1d4fbd13dd 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -195,6 +195,19 @@ 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 codec = table_props.avro_compression_codec.clone(); + let level = table_props.avro_compression_level; + let builder = ManifestWriterBuilder::new( output_file, Some(self.snapshot_id), @@ -205,7 +218,9 @@ impl<'a> SnapshotProducer<'a> { .default_partition_spec() .as_ref() .clone(), - ); + ) + .with_compression(codec, level); + match self.table.metadata().format_version() { FormatVersion::V1 => Ok(builder.build_v1()), FormatVersion::V2 => match content { @@ -386,6 +401,19 @@ 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 codec = table_props.avro_compression_codec.clone(); + let level = table_props.avro_compression_level; + let mut manifest_list_writer = match self.table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( self.table @@ -411,7 +439,8 @@ impl<'a> SnapshotProducer<'a> { next_seq_num, Some(first_row_id), ), - }; + } + .with_compression(codec, level); // Calling self.summary() before self.manifest_file() is important because self.added_data_files // will be set to an empty vec after self.manifest_file() returns, resulting in an empty summary From 091d3bc5388d20b1bd84cea29df04947d3e89530 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 13 Nov 2025 00:50:57 +0000 Subject: [PATCH 02/25] fmt --- crates/iceberg/src/spec/avro_util.rs | 33 +++++++++++++++++----- crates/iceberg/src/spec/manifest/writer.rs | 5 +++- 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index 055b304f47..9d75a35575 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -125,9 +125,11 @@ mod tests { #[test] fn test_codec_from_str_deflate_levels() { - use apache_avro::{Writer, Schema, types::Record}; 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(); @@ -151,13 +153,20 @@ mod tests { // 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"); + 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::{Writer, Schema, types::Record}; + 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(); @@ -196,12 +205,17 @@ mod tests { 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"); + assert_eq!( + encoded_100.len(), + encoded_22.len(), + "Level 100 should be clamped to 22" + ); } #[test] fn test_compression_level_differences() { - use apache_avro::{Writer, Schema, types::Record}; + 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(); @@ -225,6 +239,11 @@ mod tests { 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); + 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/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index aa727c1388..2db3ca6356 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -443,7 +443,10 @@ impl ManifestWriter { }; // Determine compression codec using helper function - let codec = codec_from_str(Some(self.compression_codec.as_str()), self.compression_level); + let codec = codec_from_str( + Some(self.compression_codec.as_str()), + self.compression_level, + ); let mut avro_writer = AvroWriter::with_codec(&avro_schema, Vec::new(), codec); avro_writer.add_user_metadata( From 41a8c1c6fe27f61e8b33d5ae7d961268261c1fb1 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 13 Nov 2025 06:54:57 +0000 Subject: [PATCH 03/25] fix clippy --- crates/iceberg/src/spec/avro_util.rs | 31 ++++++++++++++ crates/iceberg/src/spec/manifest/writer.rs | 48 ++++++++-------------- crates/iceberg/src/spec/manifest_list.rs | 43 +++++++------------ crates/iceberg/src/spec/mod.rs | 1 + crates/iceberg/src/transaction/snapshot.rs | 24 ++++++----- 5 files changed, 77 insertions(+), 70 deletions(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index 9d75a35575..12e1395d4d 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -20,6 +20,37 @@ use apache_avro::Codec; use log::warn; +/// Settings for compression codec and level. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct CompressionSettings { + /// The compression codec name (e.g., "gzip", "zstd", "deflate", "none") + pub codec: String, + /// The compression level + pub level: u8, +} + +impl CompressionSettings { + /// Create a new CompressionSettings with the specified codec and level. + pub fn new(codec: String, level: u8) -> 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 { + use crate::spec::TableProperties; + Self { + codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), + level: TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + } + } +} + /// Convert codec name and level to apache_avro::Codec. /// Returns Codec::Null for unknown or unsupported codecs. /// diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index 2db3ca6356..2ced707efd 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -28,7 +28,7 @@ use super::{ }; use crate::error::Result; use crate::io::OutputFile; -use crate::spec::avro_util::codec_from_str; +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::{ @@ -44,8 +44,7 @@ pub struct ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, - compression_codec: String, - compression_level: u8, + compression: CompressionSettings, } impl ManifestWriterBuilder { @@ -57,23 +56,19 @@ impl ManifestWriterBuilder { schema: SchemaRef, partition_spec: PartitionSpec, ) -> Self { - use crate::spec::TableProperties; - Self { output, snapshot_id, key_metadata, schema, partition_spec, - compression_codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - compression_level: TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + compression: CompressionSettings::default(), } } - /// Set compression codec and level for the manifest file. - pub fn with_compression(mut self, codec: String, level: u8) -> Self { - self.compression_codec = codec; - self.compression_level = level; + /// Set compression settings for the manifest file. + pub fn with_compression(mut self, compression: CompressionSettings) -> Self { + self.compression = compression; self } @@ -92,8 +87,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression_codec, - self.compression_level, + self.compression, ) } @@ -112,8 +106,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression_codec, - self.compression_level, + self.compression, ) } @@ -132,8 +125,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression_codec, - self.compression_level, + self.compression, ) } @@ -154,8 +146,7 @@ impl ManifestWriterBuilder { // First row id is assigned by the [`ManifestListWriter`] when the manifest // is added to the list. None, - self.compression_codec, - self.compression_level, + self.compression, ) } @@ -174,8 +165,7 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression_codec, - self.compression_level, + self.compression, ) } } @@ -202,8 +192,7 @@ pub struct ManifestWriter { metadata: ManifestMetadata, - compression_codec: String, - compression_level: u8, + compression: CompressionSettings, } impl ManifestWriter { @@ -214,8 +203,7 @@ impl ManifestWriter { key_metadata: Option>, metadata: ManifestMetadata, first_row_id: Option, - compression_codec: String, - compression_level: u8, + compression: CompressionSettings, ) -> Self { Self { output, @@ -231,8 +219,7 @@ impl ManifestWriter { key_metadata, manifest_entries: Vec::new(), metadata, - compression_codec, - compression_level, + compression, } } @@ -442,11 +429,8 @@ impl ManifestWriter { FormatVersion::V2 | FormatVersion::V3 => manifest_schema_v2(&partition_type)?, }; - // Determine compression codec using helper function - let codec = codec_from_str( - Some(self.compression_codec.as_str()), - self.compression_level, - ); + // Determine compression codec using CompressionSettings + let codec = self.compression.to_codec(); let mut avro_writer = AvroWriter::with_codec(&avro_schema, Vec::new(), codec); avro_writer.add_user_metadata( diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index fd41f33e12..e89f3a0243 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -31,7 +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::codec_from_str; +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}; @@ -99,8 +99,7 @@ pub struct ManifestListWriter { sequence_number: i64, snapshot_id: i64, next_row_id: Option, - compression_codec: String, - compression_level: u8, + compression: CompressionSettings, } impl std::fmt::Debug for ManifestListWriter { @@ -121,8 +120,6 @@ 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 { - use crate::spec::TableProperties; - let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("format-version".to_string(), "1".to_string()), @@ -140,8 +137,7 @@ impl ManifestListWriter { 0, snapshot_id, None, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + CompressionSettings::default(), ) } @@ -152,8 +148,6 @@ impl ManifestListWriter { parent_snapshot_id: Option, sequence_number: i64, ) -> Self { - use crate::spec::TableProperties; - let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("sequence-number".to_string(), sequence_number.to_string()), @@ -172,8 +166,7 @@ impl ManifestListWriter { sequence_number, snapshot_id, None, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + CompressionSettings::default(), ) } @@ -185,8 +178,6 @@ impl ManifestListWriter { sequence_number: i64, first_row_id: Option, // Always None for delete manifests ) -> Self { - use crate::spec::TableProperties; - let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("sequence-number".to_string(), sequence_number.to_string()), @@ -211,15 +202,13 @@ impl ManifestListWriter { sequence_number, snapshot_id, first_row_id, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + CompressionSettings::default(), ) } - /// Set compression codec and level for the manifest list file. - pub fn with_compression(mut self, codec: String, level: u8) -> Self { - self.compression_codec = codec.clone(); - self.compression_level = level; + /// Set compression settings for the manifest list file. + pub fn with_compression(mut self, compression: CompressionSettings) -> Self { + self.compression = compression.clone(); // Recreate the avro_writer with the new codec let avro_schema = match self.format_version { @@ -228,10 +217,10 @@ impl ManifestListWriter { FormatVersion::V3 => &MANIFEST_LIST_AVRO_SCHEMA_V3, }; - // Use helper function to get codec - let compression = codec_from_str(Some(codec.as_str()), level); + // Use CompressionSettings to get codec + let codec = compression.to_codec(); - let new_writer = Writer::with_codec(avro_schema, Vec::new(), compression); + let new_writer = Writer::with_codec(avro_schema, Vec::new(), codec); // Copy over existing metadata from the old writer // Unfortunately, we can't extract metadata from the old writer, @@ -247,8 +236,7 @@ impl ManifestListWriter { sequence_number: i64, snapshot_id: i64, first_row_id: Option, - compression_codec: String, - compression_level: u8, + compression: CompressionSettings, ) -> Self { let avro_schema = match format_version { FormatVersion::V1 => &MANIFEST_LIST_AVRO_SCHEMA_V1, @@ -256,8 +244,8 @@ impl ManifestListWriter { FormatVersion::V3 => &MANIFEST_LIST_AVRO_SCHEMA_V3, }; - // Use helper function to determine compression codec - let codec = codec_from_str(Some(compression_codec.as_str()), compression_level); + // 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 { @@ -272,8 +260,7 @@ impl ManifestListWriter { sequence_number, snapshot_id, next_row_id: first_row_id, - compression_codec, - compression_level, + compression, } } diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs index cfe9132193..bb8a50d828 100644 --- a/crates/iceberg/src/spec/mod.rs +++ b/crates/iceberg/src/spec/mod.rs @@ -38,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/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 1d4fbd13dd..c1fde6232c 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; @@ -205,8 +205,10 @@ impl<'a> SnapshotProducer<'a> { ) .with_source(e) })?; - let codec = table_props.avro_compression_codec.clone(); - let level = table_props.avro_compression_level; + let compression = CompressionSettings::new( + table_props.avro_compression_codec.clone(), + table_props.avro_compression_level, + ); let builder = ManifestWriterBuilder::new( output_file, @@ -219,7 +221,7 @@ impl<'a> SnapshotProducer<'a> { .as_ref() .clone(), ) - .with_compression(codec, level); + .with_compression(compression); match self.table.metadata().format_version() { FormatVersion::V1 => Ok(builder.build_v1()), @@ -411,8 +413,10 @@ impl<'a> SnapshotProducer<'a> { ) .with_source(e) })?; - let codec = table_props.avro_compression_codec.clone(); - let level = table_props.avro_compression_level; + let compression = CompressionSettings::new( + table_props.avro_compression_codec.clone(), + table_props.avro_compression_level, + ); let mut manifest_list_writer = match self.table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( @@ -440,7 +444,7 @@ impl<'a> SnapshotProducer<'a> { Some(first_row_id), ), } - .with_compression(codec, level); + .with_compression(compression); // Calling self.summary() before self.manifest_file() is important because self.added_data_files // will be set to an empty vec after self.manifest_file() returns, resulting in an empty summary From 51e781ea5cf85e37791965a8f81bb26496bbaf1e Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 13 Nov 2025 07:22:03 +0000 Subject: [PATCH 04/25] clippy again --- crates/iceberg/src/spec/avro_util.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index 12e1395d4d..bf1e5e545b 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -215,7 +215,7 @@ mod tests { writer.append(record).unwrap(); let encoded = writer.into_inner().unwrap(); - assert!(encoded.len() > 0, "Compression should produce output"); + assert!(!encoded.is_empty(), "Compression should produce output"); } // Test clamping - higher than 22 should be clamped to 22 From 253bf59b98d5715fa9308578980d90bfe0e30c5f Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Sun, 16 Nov 2025 00:27:42 +0000 Subject: [PATCH 05/25] wip --- crates/iceberg/src/spec/avro_util.rs | 84 +++++++++++++------- crates/iceberg/src/spec/manifest/writer.rs | 9 +-- crates/iceberg/src/spec/manifest_list.rs | 33 ++------ crates/iceberg/src/spec/table_metadata.rs | 2 +- crates/iceberg/src/spec/table_properties.rs | 86 ++++++++++++++++++--- 5 files changed, 140 insertions(+), 74 deletions(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index bf1e5e545b..3562054db1 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -23,15 +23,15 @@ use log::warn; /// Settings for compression codec and level. #[derive(Debug, Clone, PartialEq, Eq)] pub struct CompressionSettings { - /// The compression codec name (e.g., "gzip", "zstd", "deflate", "none") + /// The compression codec name (e.g., "gzip", "zstd", "deflate", "uncompressed") pub codec: String, - /// The compression level - pub level: u8, + /// The compression level (None uses codec-specific defaults: gzip=9, zstd=1) + pub level: Option, } impl CompressionSettings { /// Create a new CompressionSettings with the specified codec and level. - pub fn new(codec: String, level: u8) -> Self { + pub fn new(codec: String, level: Option) -> Self { Self { codec, level } } @@ -46,7 +46,7 @@ impl Default for CompressionSettings { use crate::spec::TableProperties; Self { codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - level: TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, + level: None, } } } @@ -56,8 +56,8 @@ impl Default for CompressionSettings { /// /// # Arguments /// -/// * `codec` - The name of the compression codec (e.g., "gzip", "zstd", "deflate", "none") -/// * `level` - The compression level. For deflate/gzip: +/// * `codec` - The name of the compression codec (e.g., "gzip", "zstd", "deflate", "uncompressed") +/// * `level` - The compression level (None uses codec defaults: gzip=9, zstd=1). For deflate/gzip: /// - 0: NoCompression /// - 1: BestSpeed /// - 9: BestCompression @@ -66,9 +66,9 @@ impl Default for CompressionSettings { /// /// # Supported Codecs /// -/// - `gzip` or `deflate`: Uses Deflate compression with specified level -/// - `zstd`: Uses Zstandard compression (level clamped to valid zstd range) -/// - `none` or `None`: No compression +/// - `gzip` or `deflate`: Uses Deflate compression with specified level (default: 9) +/// - `zstd`: Uses Zstandard compression (default: 1, level clamped to valid zstd range 0-22) +/// - `uncompressed` or `None`: No compression /// - Any other value: Defaults to no compression (Codec::Null) /// /// # Compression Levels @@ -79,16 +79,17 @@ impl Default for CompressionSettings { /// - Level 9: Best compression (slower, better compression) /// - Level 10: Uber compression (slowest, best compression) /// - Other: Default level (balanced speed/compression) -pub(crate) fn codec_from_str(codec: Option<&str>, level: u8) -> Codec { +pub(crate) fn codec_from_str(codec: Option<&str>, level: Option) -> Codec { use apache_avro::{DeflateSettings, ZstandardSettings}; match codec { Some("gzip") | Some("deflate") => { // 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 { + let compression_level = match level.unwrap_or(9) { 0 => CompressionLevel::NoCompression, 1 => CompressionLevel::BestSpeed, 9 => CompressionLevel::BestCompression, @@ -100,10 +101,11 @@ pub(crate) fn codec_from_str(codec: Option<&str>, level: u8) -> Codec { } Some("zstd") => { // Zstandard supports levels 0-22, clamp to valid range - let zstd_level = level.min(22); + // 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("none") | None => Codec::Null, + Some("uncompressed") | None => Codec::Null, Some(unknown) => { warn!( "Unrecognized compression codec '{}', using no compression (Codec::Null)", @@ -117,43 +119,65 @@ pub(crate) fn codec_from_str(codec: Option<&str>, level: u8) -> Codec { #[cfg(test)] mod tests { use super::*; + use apache_avro::{DeflateSettings, ZstandardSettings}; + use miniz_oxide::deflate::CompressionLevel; #[test] fn test_codec_from_str_gzip() { - let codec = codec_from_str(Some("gzip"), 5); - assert!(matches!(codec, Codec::Deflate(_))); + let codec = codec_from_str(Some("gzip"), Some(5)); + assert_eq!( + codec, + Codec::Deflate(DeflateSettings::new(CompressionLevel::DefaultLevel)) + ); } #[test] fn test_codec_from_str_deflate() { - let codec = codec_from_str(Some("deflate"), 9); - assert!(matches!(codec, Codec::Deflate(_))); + let codec = codec_from_str(Some("deflate"), Some(9)); + assert_eq!( + codec, + Codec::Deflate(DeflateSettings::new(CompressionLevel::BestCompression)) + ); } #[test] fn test_codec_from_str_zstd() { - let codec = codec_from_str(Some("zstd"), 3); - assert!(matches!(codec, Codec::Zstandard(_))); + let codec = codec_from_str(Some("zstd"), Some(3)); + assert_eq!(codec, Codec::Zstandard(ZstandardSettings::new(3))); } #[test] - fn test_codec_from_str_none() { - let codec = codec_from_str(Some("none"), 0); + 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, 0); + 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"), 1); + 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; @@ -170,7 +194,7 @@ mod tests { // 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"), level); + 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(); @@ -205,7 +229,7 @@ mod tests { // Test various levels by checking they produce valid codecs for level in [0, 3, 15, 22] { - let codec = codec_from_str(Some("zstd"), level); + let codec = codec_from_str(Some("zstd"), Some(level)); assert!(matches!(codec, Codec::Zstandard(_))); // Verify the codec actually works by compressing data @@ -219,8 +243,8 @@ mod tests { } // Test clamping - higher than 22 should be clamped to 22 - let codec_100 = codec_from_str(Some("zstd"), 100); - let codec_22 = codec_from_str(Some("zstd"), 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); @@ -255,14 +279,14 @@ mod tests { let test_str = "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"; // Test gzip level 0 (no compression) vs level 9 (best compression) - let codec_0 = codec_from_str(Some("gzip"), 0); + 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"), 9); + 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); diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index 2ced707efd..5db54d27cb 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -55,6 +55,7 @@ impl ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, + compression_settings : CompressionSettings ) -> Self { Self { output, @@ -62,16 +63,10 @@ impl ManifestWriterBuilder { key_metadata, schema, partition_spec, - compression: CompressionSettings::default(), + compression_settings, } } - /// Set compression settings for the manifest file. - pub fn with_compression(mut self, compression: CompressionSettings) -> Self { - self.compression = compression; - self - } - /// Build a [`ManifestWriter`] for format version 1. pub fn build_v1(self) -> ManifestWriter { let metadata = ManifestMetadata::builder() diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index e89f3a0243..f1096ccfaf 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -119,7 +119,7 @@ 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()), @@ -137,7 +137,7 @@ impl ManifestListWriter { 0, snapshot_id, None, - CompressionSettings::default(), + compression_settings, ) } @@ -147,6 +147,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()), @@ -166,7 +167,7 @@ impl ManifestListWriter { sequence_number, snapshot_id, None, - CompressionSettings::default(), + compression_settings, ) } @@ -177,6 +178,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()), @@ -202,33 +204,10 @@ impl ManifestListWriter { sequence_number, snapshot_id, first_row_id, - CompressionSettings::default(), + compression_settings, ) } - /// Set compression settings for the manifest list file. - pub fn with_compression(mut self, compression: CompressionSettings) -> Self { - self.compression = compression.clone(); - - // Recreate the avro_writer with the new codec - let avro_schema = match self.format_version { - FormatVersion::V1 => &MANIFEST_LIST_AVRO_SCHEMA_V1, - FormatVersion::V2 => &MANIFEST_LIST_AVRO_SCHEMA_V2, - FormatVersion::V3 => &MANIFEST_LIST_AVRO_SCHEMA_V3, - }; - - // Use CompressionSettings to get codec - let codec = compression.to_codec(); - - let new_writer = Writer::with_codec(avro_schema, Vec::new(), codec); - - // Copy over existing metadata from the old writer - // Unfortunately, we can't extract metadata from the old writer, - // so we'll need to handle this differently - self.avro_writer = new_writer; - self - } - fn new( format_version: FormatVersion, output_file: OutputFile, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 24c0fb99a5..9407555f2c 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -500,7 +500,7 @@ impl TableMetadata { (compressed_data, new_location) } - "none" | "" => (json_data, metadata_location.as_ref().to_string()), + "uncompressed" | "" => (json_data, metadata_location.as_ref().to_string()), other => { return Err(Error::new( ErrorKind::DataInvalid, diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index 10908362ca..cad82a95f6 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -34,6 +34,25 @@ where }) } +// Helper function to parse an optional property from a HashMap +// If the property is not found, returns None +fn parse_optional_property( + properties: &HashMap, + key: &str, +) -> Result, anyhow::Error> +where + ::Err: std::fmt::Display, +{ + properties + .get(key) + .map(|value| { + value + .parse::() + .map_err(|e| anyhow::anyhow!("Invalid value for {key}: {e}")) + }) + .transpose() +} + /// TableProperties that contains the properties of a table. #[derive(Debug)] pub struct TableProperties { @@ -53,8 +72,8 @@ pub struct TableProperties { pub metadata_compression_codec: String, /// Compression codec for Avro files (manifests, manifest lists) pub avro_compression_codec: String, - /// Compression level for Avro files - pub avro_compression_level: u8, + /// Compression level for Avro files (None uses codec-specific defaults: gzip=9, zstd=1) + pub avro_compression_level: Option, } impl TableProperties { @@ -146,8 +165,8 @@ impl TableProperties { /// Compression codec for metadata files (JSON) pub const PROPERTY_METADATA_COMPRESSION_CODEC: &str = "write.metadata.compression-codec"; - /// Default metadata compression codec - none - pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "none"; + /// Default metadata compression codec - uncompressed + pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "uncompressed"; /// Compression codec for Avro files (manifests, manifest lists) pub const PROPERTY_AVRO_COMPRESSION_CODEC: &str = "write.avro.compression-codec"; @@ -156,8 +175,8 @@ impl TableProperties { /// Compression level for Avro files pub const PROPERTY_AVRO_COMPRESSION_LEVEL: &str = "write.avro.compression-level"; - /// Default Avro compression level (9 = BestCompression) - pub const PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT: u8 = 9; + /// Default Avro compression level (None, uses codec-specific defaults: gzip=9, zstd=1) + pub const PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT: Option = None; } impl TryFrom<&HashMap> for TableProperties { @@ -206,10 +225,9 @@ impl TryFrom<&HashMap> for TableProperties { TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC, TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), )?, - avro_compression_level: parse_property( + avro_compression_level: parse_optional_property( props, TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL, - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT, )?, }) } @@ -277,7 +295,7 @@ mod tests { let table_properties = TableProperties::try_from(&props).unwrap(); assert_eq!(table_properties.metadata_compression_codec, "gzip"); assert_eq!(table_properties.avro_compression_codec, "zstd"); - assert_eq!(table_properties.avro_compression_level, 3); + assert_eq!(table_properties.avro_compression_level, Some(3)); } #[test] @@ -352,4 +370,54 @@ mod tests { "Invalid value for write.target-file-size-bytes: invalid digit found in string" )); } + + #[test] + fn test_parse_optional_property() { + // Test when key is not present - should return None + let props = HashMap::new(); + let result: Option = parse_optional_property(&props, "missing-key").unwrap(); + assert_eq!(result, None); + + // Test when key is present with valid value - should return Some(value) + let props = HashMap::from([("test-key".to_string(), "42".to_string())]); + let result: Option = parse_optional_property(&props, "test-key").unwrap(); + assert_eq!(result, Some(42)); + + // Test when key is present with invalid value - should return error + let props = HashMap::from([("test-key".to_string(), "invalid".to_string())]); + let result = parse_optional_property::(&props, "test-key"); + assert!(result.is_err()); + assert!(result + .unwrap_err() + .to_string() + .contains("Invalid value for test-key")); + } + + #[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")); + } } From 8bdb52dbf37de9aee8f537eebaf12638d07fd402 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Sun, 16 Nov 2025 00:41:10 +0000 Subject: [PATCH 06/25] address comments --- crates/iceberg/src/io/object_cache.rs | 7 ++- crates/iceberg/src/scan/mod.rs | 10 ++- crates/iceberg/src/spec/avro_util.rs | 8 ++- crates/iceberg/src/spec/manifest/mod.rs | 10 ++- crates/iceberg/src/spec/manifest/writer.rs | 5 +- crates/iceberg/src/spec/manifest_list.rs | 67 ++++++++++++++++----- crates/iceberg/src/spec/table_properties.rs | 20 +++--- crates/iceberg/src/transaction/snapshot.rs | 10 +-- 8 files changed, 101 insertions(+), 36 deletions(-) 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 index 3562054db1..19221c4c9f 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -118,10 +118,11 @@ pub(crate) fn codec_from_str(codec: Option<&str>, level: Option) -> Codec { #[cfg(test)] mod tests { - use super::*; use apache_avro::{DeflateSettings, ZstandardSettings}; use miniz_oxide::deflate::CompressionLevel; + use super::*; + #[test] fn test_codec_from_str_gzip() { let codec = codec_from_str(Some("gzip"), Some(5)); @@ -168,7 +169,10 @@ mod tests { 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))); + assert_eq!( + codec, + Codec::Deflate(DeflateSettings::new(CompressionLevel::BestCompression)) + ); } #[test] 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 5db54d27cb..db9853cc35 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -55,7 +55,7 @@ impl ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, - compression_settings : CompressionSettings + compression: CompressionSettings, ) -> Self { Self { output, @@ -63,7 +63,7 @@ impl ManifestWriterBuilder { key_metadata, schema, partition_spec, - compression_settings, + compression, } } @@ -706,6 +706,7 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), + CompressionSettings::default(), ) .build_v2_data(); writer.add_entry(entries[0].clone()).unwrap(); diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index f1096ccfaf..54dbc987e2 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -99,7 +99,6 @@ pub struct ManifestListWriter { sequence_number: i64, snapshot_id: i64, next_row_id: Option, - compression: CompressionSettings, } impl std::fmt::Debug for ManifestListWriter { @@ -119,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, compression_settings : CompressionSettings) -> 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()), @@ -147,7 +151,7 @@ impl ManifestListWriter { snapshot_id: i64, parent_snapshot_id: Option, sequence_number: i64, - compression_settings : CompressionSettings + compression_settings: CompressionSettings, ) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), @@ -178,7 +182,7 @@ impl ManifestListWriter { parent_snapshot_id: Option, sequence_number: i64, first_row_id: Option, // Always None for delete manifests - compression_settings : CompressionSettings + compression_settings: CompressionSettings, ) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), @@ -239,7 +243,6 @@ impl ManifestListWriter { sequence_number, snapshot_id, next_row_id: first_row_id, - compression, } } @@ -1381,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] @@ -1420,6 +1423,7 @@ mod test { file_io.new_output(full_path.clone()).unwrap(), 1646658105718557341, Some(1646658105718557341), + CompressionSettings::default(), ); writer @@ -1493,6 +1497,7 @@ mod test { 1646658105718557341, Some(1646658105718557341), 1, + CompressionSettings::default(), ); writer @@ -1567,6 +1572,7 @@ mod test { Some(377075049360453639), 1, Some(10), + CompressionSettings::default(), ); writer @@ -1703,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(); @@ -1750,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(); @@ -1798,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(); @@ -1846,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(); @@ -1891,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(); @@ -1938,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(); diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index cad82a95f6..df477c6f73 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -387,10 +387,12 @@ mod tests { let props = HashMap::from([("test-key".to_string(), "invalid".to_string())]); let result = parse_optional_property::(&props, "test-key"); assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains("Invalid value for test-key")); + assert!( + result + .unwrap_err() + .to_string() + .contains("Invalid value for test-key") + ); } #[test] @@ -415,9 +417,11 @@ mod tests { )]); let result = TableProperties::try_from(&props); assert!(result.is_err()); - assert!(result - .unwrap_err() - .to_string() - .contains("Invalid value for write.avro.compression-level")); + 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 c1fde6232c..b0f4276338 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -220,8 +220,8 @@ impl<'a> SnapshotProducer<'a> { .default_partition_spec() .as_ref() .clone(), - ) - .with_compression(compression); + compression, + ); match self.table.metadata().format_version() { FormatVersion::V1 => Ok(builder.build_v1()), @@ -425,6 +425,7 @@ impl<'a> SnapshotProducer<'a> { .new_output(manifest_list_path.clone())?, self.snapshot_id, self.table.metadata().current_snapshot_id(), + compression.clone(), ), FormatVersion::V2 => ManifestListWriter::v2( self.table @@ -433,6 +434,7 @@ impl<'a> SnapshotProducer<'a> { self.snapshot_id, self.table.metadata().current_snapshot_id(), next_seq_num, + compression.clone(), ), FormatVersion::V3 => ManifestListWriter::v3( self.table @@ -442,9 +444,9 @@ impl<'a> SnapshotProducer<'a> { self.table.metadata().current_snapshot_id(), next_seq_num, Some(first_row_id), + compression, ), - } - .with_compression(compression); + }; // Calling self.summary() before self.manifest_file() is important because self.added_data_files // will be set to an empty vec after self.manifest_file() returns, resulting in an empty summary From 9d27116b75672544c3ee295123f9b63f693225bb Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Sun, 16 Nov 2025 00:52:36 +0000 Subject: [PATCH 07/25] no clone needed --- crates/iceberg/src/transaction/snapshot.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index b0f4276338..74dca8927b 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -425,7 +425,7 @@ impl<'a> SnapshotProducer<'a> { .new_output(manifest_list_path.clone())?, self.snapshot_id, self.table.metadata().current_snapshot_id(), - compression.clone(), + compression, ), FormatVersion::V2 => ManifestListWriter::v2( self.table @@ -434,7 +434,7 @@ impl<'a> SnapshotProducer<'a> { self.snapshot_id, self.table.metadata().current_snapshot_id(), next_seq_num, - compression.clone(), + compression, ), FormatVersion::V3 => ManifestListWriter::v3( self.table From d1ee0b26009625f06812de98741f50d279b2df25 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Sun, 16 Nov 2025 01:22:27 +0000 Subject: [PATCH 08/25] test compression works --- crates/iceberg/src/spec/manifest/writer.rs | 128 +++++++++++++++++++++ crates/iceberg/src/spec/manifest_list.rs | 92 +++++++++++++++ 2 files changed, 220 insertions(+) diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index db9853cc35..ccea427a0b 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -726,4 +726,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 54dbc987e2..a5be5c2198 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -2096,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); + } } From 393622f6412697158e9413155f184214d651dc33 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Mon, 17 Nov 2025 21:51:19 +0000 Subject: [PATCH 09/25] comments --- Cargo.toml | 2 +- crates/iceberg/src/spec/avro_util.rs | 24 ++++++++++----------- crates/iceberg/src/spec/manifest/writer.rs | 1 - crates/iceberg/src/spec/table_metadata.rs | 6 ++++-- crates/iceberg/src/spec/table_properties.rs | 2 +- crates/iceberg/src/transaction/snapshot.rs | 4 ++-- 6 files changed, 20 insertions(+), 19 deletions(-) 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/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index 19221c4c9f..5c90c3b94a 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -23,7 +23,7 @@ use log::warn; /// Settings for compression codec and level. #[derive(Debug, Clone, PartialEq, Eq)] pub struct CompressionSettings { - /// The compression codec name (e.g., "gzip", "zstd", "deflate", "uncompressed") + /// The compression codec name (e.g., "gzip", "zstd", "snappy", "uncompressed") pub codec: String, /// The compression level (None uses codec-specific defaults: gzip=9, zstd=1) pub level: Option, @@ -56,8 +56,8 @@ impl Default for CompressionSettings { /// /// # Arguments /// -/// * `codec` - The name of the compression codec (e.g., "gzip", "zstd", "deflate", "uncompressed") -/// * `level` - The compression level (None uses codec defaults: gzip=9, zstd=1). For deflate/gzip: +/// * `codec` - The name of the compression codec (e.g., "gzip", "zstd", "snappy", "uncompressed") +/// * `level` - The compression level (None uses codec defaults: gzip=9, zstd=1). For gzip: /// - 0: NoCompression /// - 1: BestSpeed /// - 9: BestCompression @@ -66,8 +66,9 @@ impl Default for CompressionSettings { /// /// # Supported Codecs /// -/// - `gzip` or `deflate`: Uses Deflate compression with specified level (default: 9) +/// - `gzip`: Uses Deflate compression with specified level (default: 9) /// - `zstd`: Uses Zstandard compression (default: 1, 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) /// @@ -82,8 +83,9 @@ impl Default for CompressionSettings { pub(crate) fn codec_from_str(codec: Option<&str>, level: Option) -> Codec { use apache_avro::{DeflateSettings, ZstandardSettings}; - match codec { - Some("gzip") | Some("deflate") => { + // 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 @@ -105,6 +107,7 @@ pub(crate) fn codec_from_str(codec: Option<&str>, level: Option) -> Codec { 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!( @@ -133,12 +136,9 @@ mod tests { } #[test] - fn test_codec_from_str_deflate() { - let codec = codec_from_str(Some("deflate"), Some(9)); - assert_eq!( - codec, - Codec::Deflate(DeflateSettings::new(CompressionLevel::BestCompression)) - ); + fn test_codec_from_str_snappy() { + let codec = codec_from_str(Some("snappy"), None); + assert_eq!(codec, Codec::Snappy); } #[test] diff --git a/crates/iceberg/src/spec/manifest/writer.rs b/crates/iceberg/src/spec/manifest/writer.rs index ccea427a0b..672e58407f 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -424,7 +424,6 @@ impl ManifestWriter { FormatVersion::V2 | FormatVersion::V3 => manifest_schema_v2(&partition_type)?, }; - // Determine compression codec using CompressionSettings let codec = self.compression.to_codec(); let mut avro_writer = AvroWriter::with_codec(&avro_schema, Vec::new(), codec); diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 9407555f2c..451002d1f4 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -474,7 +474,9 @@ impl TableMetadata { .map(|s| s.as_str()) .unwrap_or(crate::spec::table_properties::TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT); - let (data_to_write, actual_location) = match codec { + // Use case-insensitive comparison to match Java implementation + let codec_lower = codec.to_lowercase(); + let (data_to_write, actual_location) = match codec_lower.as_str() { "gzip" => { let mut encoder = GzEncoder::new(Vec::new(), flate2::Compression::default()); encoder.write_all(&json_data).map_err(|e| { @@ -500,7 +502,7 @@ impl TableMetadata { (compressed_data, new_location) } - "uncompressed" | "" => (json_data, metadata_location.as_ref().to_string()), + "none" | "" => (json_data, metadata_location.as_ref().to_string()), other => { return Err(Error::new( ErrorKind::DataInvalid, diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index df477c6f73..0a30bae119 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -166,7 +166,7 @@ impl TableProperties { /// Compression codec for metadata files (JSON) pub const PROPERTY_METADATA_COMPRESSION_CODEC: &str = "write.metadata.compression-codec"; /// Default metadata compression codec - uncompressed - pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "uncompressed"; + pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "none"; /// Compression codec for Avro files (manifests, manifest lists) pub const PROPERTY_AVRO_COMPRESSION_CODEC: &str = "write.avro.compression-codec"; diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index 74dca8927b..5a3414ac02 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -206,7 +206,7 @@ impl<'a> SnapshotProducer<'a> { .with_source(e) })?; let compression = CompressionSettings::new( - table_props.avro_compression_codec.clone(), + table_props.avro_compression_codec, table_props.avro_compression_level, ); @@ -414,7 +414,7 @@ impl<'a> SnapshotProducer<'a> { .with_source(e) })?; let compression = CompressionSettings::new( - table_props.avro_compression_codec.clone(), + table_props.avro_compression_codec, table_props.avro_compression_level, ); From 46fdb8e779bfa2e23e856098a7faed1147a50f01 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Mon, 17 Nov 2025 21:57:27 +0000 Subject: [PATCH 10/25] update tests --- crates/iceberg/src/spec/avro_util.rs | 3 ++- crates/iceberg/src/spec/table_metadata.rs | 12 ++---------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index 5c90c3b94a..c9f3cfffe0 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -128,7 +128,8 @@ mod tests { #[test] fn test_codec_from_str_gzip() { - let codec = codec_from_str(Some("gzip"), Some(5)); + // 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)) diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 451002d1f4..1d02c924e4 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -3643,11 +3643,11 @@ mod tests { // Get a test metadata and add gzip compression property let original_metadata: TableMetadata = get_test_table_metadata("TableMetadataV2Valid.json"); - // Modify properties to enable gzip compression + // Modify properties to enable gzip compression (using mixed case to test case-insensitive matching) let mut props = original_metadata.properties.clone(); props.insert( TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), - "gzip".to_string(), + "GziP".to_string(), ); // Use builder to create new metadata with updated properties let compressed_metadata = @@ -3683,14 +3683,6 @@ mod tests { .await .unwrap(); - // Verify the properties include the compression codec - assert_eq!( - read_metadata - .properties - .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC), - Some(&"gzip".to_string()) - ); - // Verify the complete round-trip: read metadata should match what we wrote assert_eq!(read_metadata, compressed_metadata); } From ec96917cd59ee5f419553fa4db74114fd1af9dd8 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Wed, 19 Nov 2025 19:06:53 +0000 Subject: [PATCH 11/25] address comments --- crates/iceberg/src/spec/avro_util.rs | 3 ++- crates/iceberg/src/spec/table_metadata.rs | 12 +++++------ crates/iceberg/src/spec/table_properties.rs | 22 ++++++++++++--------- 3 files changed, 20 insertions(+), 17 deletions(-) diff --git a/crates/iceberg/src/spec/avro_util.rs b/crates/iceberg/src/spec/avro_util.rs index c9f3cfffe0..8aaca29fd6 100644 --- a/crates/iceberg/src/spec/avro_util.rs +++ b/crates/iceberg/src/spec/avro_util.rs @@ -20,6 +20,8 @@ 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 { @@ -43,7 +45,6 @@ impl CompressionSettings { impl Default for CompressionSettings { fn default() -> Self { - use crate::spec::TableProperties; Self { codec: TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), level: None, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 1d02c924e4..9fa70c2df0 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -22,12 +22,13 @@ use std::cmp::Ordering; use std::collections::HashMap; use std::fmt::{Display, Formatter}; use std::hash::Hash; -use std::io::Read as _; +use std::io::{Read as _, Write as _}; use std::sync::Arc; use _serde::TableMetadataEnum; use chrono::{DateTime, Utc}; use flate2::read::GzDecoder; +use flate2::write::GzEncoder; use serde::{Deserialize, Serialize}; use serde_repr::{Deserialize_repr, Serialize_repr}; use uuid::Uuid; @@ -37,6 +38,7 @@ pub use super::table_metadata_builder::{TableMetadataBuildResult, TableMetadataB use super::{ DEFAULT_PARTITION_SPEC_ID, PartitionSpecRef, PartitionStatisticsFile, SchemaId, SchemaRef, SnapshotRef, SnapshotRetention, SortOrder, SortOrderRef, StatisticsFile, StructType, + TableProperties, }; use crate::error::{Result, timestamp_ms_to_utc}; use crate::io::FileIO; @@ -461,18 +463,14 @@ impl TableMetadata { file_io: &FileIO, metadata_location: impl AsRef, ) -> Result<()> { - use std::io::Write as _; - - use flate2::write::GzEncoder; - let json_data = serde_json::to_vec(self)?; // Check if compression is enabled via table properties let codec = self .properties - .get(crate::spec::table_properties::TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) .map(|s| s.as_str()) - .unwrap_or(crate::spec::table_properties::TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT); + .unwrap_or(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT); // Use case-insensitive comparison to match Java implementation let codec_lower = codec.to_lowercase(); diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index 0a30bae119..1a600ba16e 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -16,39 +16,43 @@ // 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}")) + .map_err(|e| Error::new(ErrorKind::DataInvalid, format!("Invalid value for {key}: {e}"))) }) } // Helper function to parse an optional property from a HashMap // If the property is not found, returns None -fn parse_optional_property( +fn parse_optional_property( properties: &HashMap, key: &str, -) -> Result, anyhow::Error> +) -> crate::error::Result> where - ::Err: std::fmt::Display, + ::Err: Display, { properties .get(key) .map(|value| { value .parse::() - .map_err(|e| anyhow::anyhow!("Invalid value for {key}: {e}")) + .map_err(|e| Error::new(ErrorKind::DataInvalid, format!("Invalid value for {key}: {e}"))) }) .transpose() } @@ -181,7 +185,7 @@ impl TableProperties { 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 { From 46efa9e5b55cb21b0bf803a754e434420009e936 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Wed, 19 Nov 2025 21:16:41 +0000 Subject: [PATCH 12/25] Remove Avro compression changes, keep only JSON metadata compression --- Cargo.lock | 2 - Cargo.toml | 2 +- crates/iceberg/Cargo.toml | 2 - crates/iceberg/src/io/object_cache.rs | 7 +- crates/iceberg/src/scan/mod.rs | 10 +- crates/iceberg/src/spec/avro_util.rs | 310 -------------------- crates/iceberg/src/spec/manifest/mod.rs | 10 +- crates/iceberg/src/spec/manifest/writer.rs | 147 +--------- crates/iceberg/src/spec/manifest_list.rs | 166 +---------- crates/iceberg/src/spec/mod.rs | 2 - crates/iceberg/src/spec/table_properties.rs | 124 +------- crates/iceberg/src/transaction/snapshot.rs | 43 +-- 12 files changed, 27 insertions(+), 798 deletions(-) delete mode 100644 crates/iceberg/src/spec/avro_util.rs diff --git a/Cargo.lock b/Cargo.lock index d094e4edc7..2edc51a4ea 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3521,9 +3521,7 @@ 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 349e33112d..c10c01d94a 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", "snappy"] } +apache-avro = { version = "0.20", features = ["zstandard"] } array-init = "2" arrow-arith = "56.2" arrow-array = "56.2" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index b1fec279b2..895a5cf5e4 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -67,8 +67,6 @@ 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 36c8e3e6fd..af297bebb5 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -194,9 +194,8 @@ mod tests { use crate::TableIdent; use crate::io::{FileIO, OutputFile}; use crate::spec::{ - CompressionSettings, DataContentType, DataFileBuilder, DataFileFormat, Literal, - ManifestEntry, ManifestListWriter, ManifestStatus, ManifestWriterBuilder, Struct, - TableMetadata, + DataContentType, DataFileBuilder, DataFileFormat, Literal, ManifestEntry, + ManifestListWriter, ManifestStatus, ManifestWriterBuilder, Struct, TableMetadata, }; use crate::table::Table; @@ -276,7 +275,6 @@ mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), - CompressionSettings::default(), ) .build_v2_data(); writer @@ -309,7 +307,6 @@ 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 85009243b1..6884e00b9b 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::{ - CompressionSettings, DataContentType, DataFileBuilder, DataFileFormat, Datum, Literal, - ManifestEntry, ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, - PartitionSpec, PrimitiveType, Schema, Struct, StructType, TableMetadata, Type, + DataContentType, DataFileBuilder, DataFileFormat, Datum, Literal, ManifestEntry, + ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, PartitionSpec, + PrimitiveType, Schema, Struct, StructType, TableMetadata, Type, }; use crate::table::Table; @@ -749,7 +749,6 @@ pub mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), - CompressionSettings::default(), ) .build_v2_data(); writer @@ -827,7 +826,6 @@ 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()) @@ -963,7 +961,6 @@ pub mod tests { None, current_schema.clone(), current_partition_spec.as_ref().clone(), - CompressionSettings::default(), ) .build_v2_data(); @@ -1048,7 +1045,6 @@ 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 deleted file mode 100644 index 8aaca29fd6..0000000000 --- a/crates/iceberg/src/spec/avro_util.rs +++ /dev/null @@ -1,310 +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. - -//! 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: gzip=9, zstd=1) - 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 (None uses codec defaults: gzip=9, zstd=1). For gzip: -/// - 0: NoCompression -/// - 1: BestSpeed -/// - 9: BestCompression -/// - 10: UberCompression -/// - Other values: DefaultLevel (6) -/// -/// # Supported Codecs -/// -/// - `gzip`: Uses Deflate compression with specified level (default: 9) -/// - `zstd`: Uses Zstandard compression (default: 1, 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) -/// -/// # Compression Levels -/// -/// The compression level mapping is based on miniz_oxide's CompressionLevel enum: -/// - Level 0: No compression -/// - Level 1: Best speed (fastest) -/// - Level 9: Best compression (slower, better compression) -/// - Level 10: Uber compression (slowest, best compression) -/// - Other: Default level (balanced speed/compression) -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 e3679433eb..51219bfdb7 100644 --- a/crates/iceberg/src/spec/manifest/mod.rs +++ b/crates/iceberg/src/spec/manifest/mod.rs @@ -166,9 +166,7 @@ mod tests { use super::*; use crate::io::FileIOBuilder; - use crate::spec::{ - CompressionSettings, Literal, NestedField, PrimitiveType, Struct, Transform, Type, - }; + use crate::spec::{Literal, NestedField, PrimitiveType, Struct, Transform, Type}; #[tokio::test] async fn test_parse_manifest_v2_unpartition() { @@ -274,7 +272,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -460,7 +457,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -558,7 +554,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v1(); for entry in &entries { @@ -668,7 +663,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v1(); for entry in &entries { @@ -777,7 +771,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v2_data(); for entry in &entries { @@ -1057,7 +1050,6 @@ 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 672e58407f..ebb0590bcf 100644 --- a/crates/iceberg/src/spec/manifest/writer.rs +++ b/crates/iceberg/src/spec/manifest/writer.rs @@ -28,7 +28,6 @@ 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::{ @@ -44,7 +43,6 @@ pub struct ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, - compression: CompressionSettings, } impl ManifestWriterBuilder { @@ -55,7 +53,6 @@ impl ManifestWriterBuilder { key_metadata: Option>, schema: SchemaRef, partition_spec: PartitionSpec, - compression: CompressionSettings, ) -> Self { Self { output, @@ -63,7 +60,6 @@ impl ManifestWriterBuilder { key_metadata, schema, partition_spec, - compression, } } @@ -82,7 +78,6 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression, ) } @@ -101,7 +96,6 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression, ) } @@ -120,7 +114,6 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression, ) } @@ -141,7 +134,6 @@ impl ManifestWriterBuilder { // First row id is assigned by the [`ManifestListWriter`] when the manifest // is added to the list. None, - self.compression, ) } @@ -160,7 +152,6 @@ impl ManifestWriterBuilder { self.key_metadata, metadata, None, - self.compression, ) } } @@ -186,8 +177,6 @@ pub struct ManifestWriter { manifest_entries: Vec, metadata: ManifestMetadata, - - compression: CompressionSettings, } impl ManifestWriter { @@ -198,7 +187,6 @@ impl ManifestWriter { key_metadata: Option>, metadata: ManifestMetadata, first_row_id: Option, - compression: CompressionSettings, ) -> Self { Self { output, @@ -214,7 +202,6 @@ impl ManifestWriter { key_metadata, manifest_entries: Vec::new(), metadata, - compression, } } @@ -423,10 +410,7 @@ impl ManifestWriter { // Manifest schema did not change between V2 and V3 FormatVersion::V2 | FormatVersion::V3 => manifest_schema_v2(&partition_type)?, }; - - let codec = self.compression.to_codec(); - - let mut avro_writer = AvroWriter::with_codec(&avro_schema, Vec::new(), codec); + 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| { @@ -705,7 +689,6 @@ mod tests { None, metadata.schema.clone(), metadata.partition_spec.clone(), - CompressionSettings::default(), ) .build_v2_data(); writer.add_entry(entries[0].clone()).unwrap(); @@ -725,132 +708,4 @@ 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 a5be5c2198..5e97e5466e 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -31,7 +31,6 @@ 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}; @@ -118,12 +117,7 @@ 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, - compression_settings: CompressionSettings, - ) -> Self { + pub fn v1(output_file: OutputFile, snapshot_id: i64, parent_snapshot_id: Option) -> Self { let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), ("format-version".to_string(), "1".to_string()), @@ -141,7 +135,6 @@ impl ManifestListWriter { 0, snapshot_id, None, - compression_settings, ) } @@ -151,7 +144,6 @@ 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()), @@ -171,7 +163,6 @@ impl ManifestListWriter { sequence_number, snapshot_id, None, - compression_settings, ) } @@ -182,7 +173,6 @@ 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()), @@ -208,7 +198,6 @@ impl ManifestListWriter { sequence_number, snapshot_id, first_row_id, - compression_settings, ) } @@ -219,18 +208,13 @@ 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, }; - - // Use CompressionSettings to determine compression codec - let codec = compression.to_codec(); - - let mut avro_writer = Writer::with_codec(avro_schema, Vec::new(), codec); + let mut avro_writer = Writer::new(avro_schema, Vec::new()); for (key, value) in metadata { avro_writer .add_user_metadata(key, value) @@ -1384,8 +1368,8 @@ mod test { use crate::io::FileIOBuilder; use crate::spec::manifest_list::_serde::{ManifestListV1, ManifestListV3}; use crate::spec::{ - CompressionSettings, Datum, FieldSummary, ManifestContentType, ManifestFile, ManifestList, - ManifestListWriter, UNASSIGNED_SEQUENCE_NUMBER, + Datum, FieldSummary, ManifestContentType, ManifestFile, ManifestList, ManifestListWriter, + UNASSIGNED_SEQUENCE_NUMBER, }; #[tokio::test] @@ -1423,7 +1407,6 @@ mod test { file_io.new_output(full_path.clone()).unwrap(), 1646658105718557341, Some(1646658105718557341), - CompressionSettings::default(), ); writer @@ -1497,7 +1480,6 @@ mod test { 1646658105718557341, Some(1646658105718557341), 1, - CompressionSettings::default(), ); writer @@ -1572,7 +1554,6 @@ mod test { Some(377075049360453639), 1, Some(10), - CompressionSettings::default(), ); writer @@ -1709,12 +1690,7 @@ 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), - CompressionSettings::default(), - ); + let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, Some(0)); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1761,13 +1737,7 @@ 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, - CompressionSettings::default(), - ); + let mut writer = ManifestListWriter::v2(output_file, snapshot_id, Some(0), seq_num); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1815,14 +1785,8 @@ 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), - CompressionSettings::default(), - ); + let mut writer = + ManifestListWriter::v3(output_file, snapshot_id, Some(0), seq_num, Some(10)); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1869,12 +1833,7 @@ 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), - CompressionSettings::default(), - ); + let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, Some(0)); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1919,12 +1878,7 @@ 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), - CompressionSettings::default(), - ); + let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, Some(0)); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1971,13 +1925,7 @@ 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, - CompressionSettings::default(), - ); + let mut writer = ManifestListWriter::v2(output_file, snapshot_id, Some(0), seq_num); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -2096,96 +2044,4 @@ 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 bb8a50d828..44b35e5a6b 100644 --- a/crates/iceberg/src/spec/mod.rs +++ b/crates/iceberg/src/spec/mod.rs @@ -17,7 +17,6 @@ //! Spec for Iceberg. -mod avro_util; mod datatypes; mod encrypted_key; mod manifest; @@ -38,7 +37,6 @@ 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 1a600ba16e..c308ad377d 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -38,25 +38,6 @@ where }) } -// Helper function to parse an optional property from a HashMap -// If the property is not found, returns None -fn parse_optional_property( - properties: &HashMap, - key: &str, -) -> crate::error::Result> -where - ::Err: Display, -{ - properties - .get(key) - .map(|value| { - value - .parse::() - .map_err(|e| Error::new(ErrorKind::DataInvalid, format!("Invalid value for {key}: {e}"))) - }) - .transpose() -} - /// TableProperties that contains the properties of a table. #[derive(Debug)] pub struct TableProperties { @@ -74,10 +55,6 @@ pub struct TableProperties { pub write_target_file_size_bytes: usize, /// Compression codec for metadata files (JSON) pub metadata_compression_codec: String, - /// Compression codec for Avro files (manifests, manifest lists) - pub avro_compression_codec: String, - /// Compression level for Avro files (None uses codec-specific defaults: gzip=9, zstd=1) - pub avro_compression_level: Option, } impl TableProperties { @@ -171,16 +148,6 @@ impl TableProperties { pub const PROPERTY_METADATA_COMPRESSION_CODEC: &str = "write.metadata.compression-codec"; /// Default metadata compression codec - uncompressed pub const PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT: &str = "none"; - - /// 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"; - /// Default Avro compression level (None, uses codec-specific defaults: gzip=9, zstd=1) - pub const PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT: Option = None; } impl TryFrom<&HashMap> for TableProperties { @@ -224,15 +191,6 @@ impl TryFrom<&HashMap> for TableProperties { TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC, TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string(), )?, - avro_compression_codec: parse_property( - props, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string(), - )?, - avro_compression_level: parse_optional_property( - props, - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL, - )?, }) } } @@ -270,36 +228,16 @@ mod tests { table_properties.metadata_compression_codec, TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string() ); - assert_eq!( - table_properties.avro_compression_codec, - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC_DEFAULT.to_string() - ); - assert_eq!( - table_properties.avro_compression_level, - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL_DEFAULT - ); } #[test] fn test_table_properties_compression() { - let props = HashMap::from([ - ( - TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), - "gzip".to_string(), - ), - ( - TableProperties::PROPERTY_AVRO_COMPRESSION_CODEC.to_string(), - "zstd".to_string(), - ), - ( - TableProperties::PROPERTY_AVRO_COMPRESSION_LEVEL.to_string(), - "3".to_string(), - ), - ]); + let props = HashMap::from([( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "gzip".to_string(), + )]); let table_properties = TableProperties::try_from(&props).unwrap(); assert_eq!(table_properties.metadata_compression_codec, "gzip"); - assert_eq!(table_properties.avro_compression_codec, "zstd"); - assert_eq!(table_properties.avro_compression_level, Some(3)); } #[test] @@ -374,58 +312,4 @@ mod tests { "Invalid value for write.target-file-size-bytes: invalid digit found in string" )); } - - #[test] - fn test_parse_optional_property() { - // Test when key is not present - should return None - let props = HashMap::new(); - let result: Option = parse_optional_property(&props, "missing-key").unwrap(); - assert_eq!(result, None); - - // Test when key is present with valid value - should return Some(value) - let props = HashMap::from([("test-key".to_string(), "42".to_string())]); - let result: Option = parse_optional_property(&props, "test-key").unwrap(); - assert_eq!(result, Some(42)); - - // Test when key is present with invalid value - should return error - let props = HashMap::from([("test-key".to_string(), "invalid".to_string())]); - let result = parse_optional_property::(&props, "test-key"); - assert!(result.is_err()); - assert!( - result - .unwrap_err() - .to_string() - .contains("Invalid value for test-key") - ); - } - - #[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 5a3414ac02..4f85962ff1 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::{ - CompressionSettings, DataFile, DataFileFormat, FormatVersion, MAIN_BRANCH, ManifestContentType, - ManifestEntry, ManifestFile, ManifestListWriter, ManifestWriter, ManifestWriterBuilder, - Operation, Snapshot, SnapshotReference, SnapshotRetention, SnapshotSummaryCollector, Struct, - StructType, Summary, TableProperties, update_snapshot_summaries, + 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,21 +195,6 @@ 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), @@ -220,9 +205,7 @@ 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 { @@ -403,21 +386,6 @@ 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 @@ -425,7 +393,6 @@ 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 @@ -434,7 +401,6 @@ impl<'a> SnapshotProducer<'a> { self.snapshot_id, self.table.metadata().current_snapshot_id(), next_seq_num, - compression, ), FormatVersion::V3 => ManifestListWriter::v3( self.table @@ -444,7 +410,6 @@ impl<'a> SnapshotProducer<'a> { self.table.metadata().current_snapshot_id(), next_seq_num, Some(first_row_id), - compression, ), }; From e96fe1acb319f77958aa890a90840af534ff07a4 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 18:19:43 +0000 Subject: [PATCH 13/25] address some comments --- .../iceberg/src/catalog/metadata_location.rs | 121 ++++++++++++++++-- crates/iceberg/src/spec/table_metadata.rs | 23 ++-- crates/iceberg/src/spec/table_properties.rs | 43 ++++--- 3 files changed, 153 insertions(+), 34 deletions(-) diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index 3705ee42dc..dacf5c1dda 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -20,24 +20,52 @@ use std::str::FromStr; use uuid::Uuid; +use crate::spec::TableProperties; use crate::{Error, ErrorKind, Result}; /// Helper for parsing a location of the format: `/metadata/-.metadata.json` +/// or with compression: `/metadata/-.gz.metadata.json` #[derive(Clone, Debug, PartialEq)] pub struct MetadataLocation { table_location: String, version: i32, id: Uuid, + compression_suffix: Option, } impl MetadataLocation { /// Creates a completely new metadata location starting at version 0. /// Only used for creating a new table. For updates, see `with_next_version`. + #[deprecated( + since = "0.7.0", + note = "Use new_with_table instead to properly handle compression settings" + )] pub fn new_with_table_location(table_location: impl ToString) -> Self { Self { table_location: table_location.to_string(), version: 0, id: Uuid::new_v4(), + compression_suffix: None, + } + } + + /// Creates a completely new metadata location starting at version 0, + /// with compression settings from the table's properties. + /// Only used for creating a new table. For updates, see `with_next_version`. + pub fn new_with_table(table_location: impl ToString, properties: &std::collections::HashMap) -> Self { + let compression_suffix = properties + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .and_then(|codec| match codec.to_lowercase().as_str() { + "gzip" => Some(".gz".to_string()), + "none" | "" => None, + _ => None, + }); + + Self { + table_location: table_location.to_string(), + version: 0, + id: Uuid::new_v4(), + compression_suffix, } } @@ -47,6 +75,7 @@ impl MetadataLocation { table_location: self.table_location.clone(), version: self.version + 1, id: Uuid::new_v4(), + compression_suffix: self.compression_suffix.clone(), } } @@ -59,30 +88,41 @@ impl MetadataLocation { Ok(prefix.to_string()) } - /// Parses a file name of the format `-.metadata.json`. - fn parse_file_name(file_name: &str) -> Result<(i32, Uuid)> { - let (version, id) = file_name + /// Parses a file name of the format `-.metadata.json` + /// or with compression: `-.gz.metadata.json`. + fn parse_file_name(file_name: &str) -> Result<(i32, Uuid, Option)> { + let stripped = file_name .strip_suffix(".metadata.json") .ok_or(Error::new( ErrorKind::Unexpected, format!("Invalid metadata file ending: {file_name}"), - ))? + ))?; + + // Check for compression suffix (e.g., .gz) + let (stripped, compression_suffix) = if let Some(s) = stripped.strip_suffix(".gz") { + (s, Some(".gz".to_string())) + } else { + (stripped, None) + }; + + let (version, id) = stripped .split_once('-') .ok_or(Error::new( ErrorKind::Unexpected, format!("Invalid metadata file name format: {file_name}"), ))?; - Ok((version.parse::()?, Uuid::parse_str(id)?)) + Ok((version.parse::()?, Uuid::parse_str(id)?, compression_suffix)) } } impl Display for MetadataLocation { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let suffix = self.compression_suffix.as_deref().unwrap_or(""); write!( f, - "{}/metadata/{:0>5}-{}.metadata.json", - self.table_location, self.version, self.id + "{}/metadata/{:0>5}-{}{}.metadata.json", + self.table_location, self.version, self.id, suffix ) } } @@ -97,12 +137,13 @@ impl FromStr for MetadataLocation { ))?; let prefix = Self::parse_metadata_path_prefix(path)?; - let (version, id) = Self::parse_file_name(file_name)?; + let (version, id, compression_suffix) = Self::parse_file_name(file_name)?; Ok(MetadataLocation { table_location: prefix, version, id, + compression_suffix, }) } } @@ -125,6 +166,7 @@ mod test { table_location: "".to_string(), version: 1234567, id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: None, }), ), // Some prefix @@ -134,6 +176,7 @@ mod test { table_location: "/abc".to_string(), version: 1234567, id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: None, }), ), // Longer prefix @@ -143,6 +186,7 @@ mod test { table_location: "/abc/def".to_string(), version: 1234567, id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: None, }), ), // Prefix with special characters @@ -152,6 +196,7 @@ mod test { table_location: "https://127.0.0.1".to_string(), version: 1234567, id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: None, }), ), // Another id @@ -161,6 +206,7 @@ mod test { table_location: "/abc".to_string(), version: 1234567, id: Uuid::from_str("81056704-ce5b-41c4-bb83-eb6408081af6").unwrap(), + compression_suffix: None, }), ), // Version 0 @@ -170,6 +216,17 @@ mod test { table_location: "/abc".to_string(), version: 0, id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: None, + }), + ), + // With gzip compression + ( + "/abc/metadata/1234567-2cd22b57-5127-4198-92ba-e4e67c79821b.gz.metadata.json", + Ok(MetadataLocation { + table_location: "/abc".to_string(), + version: 1234567, + id: Uuid::from_str("2cd22b57-5127-4198-92ba-e4e67c79821b").unwrap(), + compression_suffix: Some(".gz".to_string()), }), ), // Negative version @@ -217,6 +274,7 @@ mod test { #[test] fn test_metadata_location_with_next_version() { let test_cases = vec![ + #[allow(deprecated)] MetadataLocation::new_with_table_location("/abc"), MetadataLocation::from_str( "/abc/def/metadata/1234567-2cd22b57-5127-4198-92ba-e4e67c79821b.metadata.json", @@ -233,4 +291,51 @@ mod test { assert_ne!(next.id, input.id); } } + + #[test] + fn test_metadata_location_new_with_table() { + use std::collections::HashMap; + + // Test with no compression + let props_none = HashMap::new(); + let location = MetadataLocation::new_with_table("/test/table", &props_none); + assert_eq!(location.table_location, "/test/table"); + assert_eq!(location.version, 0); + assert_eq!(location.compression_suffix, None); + assert_eq!( + location.to_string(), + format!("/test/table/metadata/00000-{}.metadata.json", location.id) + ); + + // Test with gzip compression + let mut props_gzip = HashMap::new(); + props_gzip.insert( + "write.metadata.compression-codec".to_string(), + "gzip".to_string(), + ); + let location = MetadataLocation::new_with_table("/test/table", &props_gzip); + assert_eq!(location.compression_suffix, Some(".gz".to_string())); + assert_eq!( + location.to_string(), + format!("/test/table/metadata/00000-{}.gz.metadata.json", location.id) + ); + + // Test with "none" codec (explicitly no compression) + let mut props_explicit_none = HashMap::new(); + props_explicit_none.insert( + "write.metadata.compression-codec".to_string(), + "none".to_string(), + ); + let location = MetadataLocation::new_with_table("/test/table", &props_explicit_none); + assert_eq!(location.compression_suffix, None); + + // Test case insensitivity + let mut props_gzip_upper = HashMap::new(); + props_gzip_upper.insert( + "write.metadata.compression-codec".to_string(), + "GZIP".to_string(), + ); + let location = MetadataLocation::new_with_table("/test/table", &props_gzip_upper); + assert_eq!(location.compression_suffix, Some(".gz".to_string())); + } } diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 9fa70c2df0..06925e3dc3 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -469,13 +469,11 @@ impl TableMetadata { let codec = self .properties .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) - .map(|s| s.as_str()) - .unwrap_or(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT); + .map(|s| s.as_str()); // Use case-insensitive comparison to match Java implementation - let codec_lower = codec.to_lowercase(); - let (data_to_write, actual_location) = match codec_lower.as_str() { - "gzip" => { + let (data_to_write, actual_location) = match codec.map(|s| s.to_lowercase()).as_deref() { + Some("gzip") => { let mut encoder = GzEncoder::new(Vec::new(), flate2::Compression::default()); encoder.write_all(&json_data).map_err(|e| { Error::new( @@ -491,17 +489,22 @@ impl TableMetadata { // Modify filename to add .gz before .metadata.json let location = metadata_location.as_ref(); - let new_location = if location.ends_with(".metadata.json") { + let new_location = if location.ends_with(".gz.metadata.json") { + // File already has the correct compressed naming convention + // This check can be removed after the deprecated method for naming is removed, + // but provides safety that compressed files have the correct naming convention. + location.to_string() + } else if location.ends_with(".metadata.json") { location.replace(".metadata.json", ".gz.metadata.json") } else { - // If it doesn't end with expected pattern, just append .gz - format!("{}.gz", location) + // Location doesn't end with expected pattern, use as-is + location.to_string() }; (compressed_data, new_location) } - "none" | "" => (json_data, metadata_location.as_ref().to_string()), - other => { + None | Some("none") | Some("") => (json_data, metadata_location.as_ref().to_string()), + Some(other) => { return Err(Error::new( ErrorKind::DataInvalid, format!("Unsupported metadata compression codec: {}", other), diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index c308ad377d..aaa56408b7 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -19,7 +19,7 @@ use std::collections::HashMap; use std::fmt::Display; use std::str::FromStr; -use crate::error::{Error, ErrorKind}; +use crate::error::{Error, ErrorKind, Result}; // Helper function to parse a property from a HashMap // If the property is not found, use the default value @@ -27,7 +27,7 @@ fn parse_property( properties: &HashMap, key: &str, default: T, -) -> crate::error::Result +) -> Result where ::Err: Display, { @@ -53,8 +53,8 @@ pub struct TableProperties { pub write_format_default: String, /// The target file size for files. pub write_target_file_size_bytes: usize, - /// Compression codec for metadata files (JSON) - pub metadata_compression_codec: String, + /// Compression codec for metadata files (JSON), None means no compression + pub metadata_compression_codec: Option, } impl TableProperties { @@ -152,9 +152,9 @@ impl TableProperties { impl TryFrom<&HashMap> for TableProperties { // parse by entry key or use default value - type Error = crate::Error; + type Error = Error; - fn try_from(props: &HashMap) -> Result { + fn try_from(props: &HashMap) -> Result { Ok(TableProperties { commit_num_retries: parse_property( props, @@ -186,11 +186,12 @@ impl TryFrom<&HashMap> for TableProperties { TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, )?, - metadata_compression_codec: parse_property( - props, - TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC, - TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string(), - )?, + metadata_compression_codec: props + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .and_then(|v| match v.as_str() { + "none" | "" => None, + codec => Some(codec.to_string()), + }), }) } } @@ -223,21 +224,31 @@ mod tests { table_properties.write_target_file_size_bytes, TableProperties::PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT ); - // Test compression defaults + // Test compression defaults (none means None) + assert_eq!(table_properties.metadata_compression_codec, None); + } + + #[test] + fn test_table_properties_compression() { + let props = HashMap::from([( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "gzip".to_string(), + )]); + let table_properties = TableProperties::try_from(&props).unwrap(); assert_eq!( table_properties.metadata_compression_codec, - TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC_DEFAULT.to_string() + Some("gzip".to_string()) ); } #[test] - fn test_table_properties_compression() { + fn test_table_properties_compression_none() { let props = HashMap::from([( TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), - "gzip".to_string(), + "none".to_string(), )]); let table_properties = TableProperties::try_from(&props).unwrap(); - assert_eq!(table_properties.metadata_compression_codec, "gzip"); + assert_eq!(table_properties.metadata_compression_codec, None); } #[test] From 53d09975b2cdb4584a38e0163f5edb6d8ada5b87 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 18:28:28 +0000 Subject: [PATCH 14/25] don't reference deprecated method --- crates/catalog/glue/src/catalog.rs | 2 +- crates/catalog/glue/src/utils.rs | 2 +- crates/catalog/hms/src/catalog.rs | 2 +- crates/catalog/hms/src/utils.rs | 4 +- crates/catalog/s3tables/src/catalog.rs | 4 +- crates/catalog/sql/src/catalog.rs | 3 +- crates/iceberg/src/catalog/memory/catalog.rs | 3 +- .../iceberg/src/catalog/metadata_location.rs | 37 +++++++++++-------- crates/iceberg/src/spec/table_properties.rs | 9 +++-- 9 files changed, 40 insertions(+), 26 deletions(-) diff --git a/crates/catalog/glue/src/catalog.rs b/crates/catalog/glue/src/catalog.rs index 4514f2d7ab..21a1ed1b28 100644 --- a/crates/catalog/glue/src/catalog.rs +++ b/crates/catalog/glue/src/catalog.rs @@ -471,7 +471,7 @@ impl Catalog for GlueCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_table_location(location.clone()).to_string(); + MetadataLocation::new_with_table(location.clone(), &metadata.properties).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/glue/src/utils.rs b/crates/catalog/glue/src/utils.rs index 457471b34a..50777d7cf4 100644 --- a/crates/catalog/glue/src/utils.rs +++ b/crates/catalog/glue/src/utils.rs @@ -306,8 +306,8 @@ mod tests { fn test_convert_to_glue_table() -> Result<()> { let table_name = "my_table".to_string(); let location = "s3a://warehouse/hive".to_string(); - let metadata_location = MetadataLocation::new_with_table_location(location).to_string(); let properties = HashMap::new(); + let metadata_location = MetadataLocation::new_with_table(location, &properties).to_string(); let schema = Schema::builder() .with_schema_id(1) .with_fields(vec![ diff --git a/crates/catalog/hms/src/catalog.rs b/crates/catalog/hms/src/catalog.rs index b7d192210b..5c5d1b89b0 100644 --- a/crates/catalog/hms/src/catalog.rs +++ b/crates/catalog/hms/src/catalog.rs @@ -443,7 +443,7 @@ impl Catalog for HmsCatalog { .metadata; let metadata_location = - MetadataLocation::new_with_table_location(location.clone()).to_string(); + MetadataLocation::new_with_table(location.clone(), &metadata.properties).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/hms/src/utils.rs b/crates/catalog/hms/src/utils.rs index 096e792f61..cee1881645 100644 --- a/crates/catalog/hms/src/utils.rs +++ b/crates/catalog/hms/src/utils.rs @@ -343,9 +343,9 @@ mod tests { let db_name = "my_db".to_string(); let table_name = "my_table".to_string(); let location = "s3a://warehouse/hms".to_string(); - let metadata_location = - MetadataLocation::new_with_table_location(location.clone()).to_string(); let properties = HashMap::new(); + let metadata_location = + MetadataLocation::new_with_table(location.clone(), &properties).to_string(); let schema = Schema::builder() .with_schema_id(1) .with_fields(vec![ diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index 3606fac99a..161257c2a7 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -468,7 +468,7 @@ impl Catalog for S3TablesCatalog { .await .map_err(from_aws_sdk_error)?; let warehouse_location = get_resp.warehouse_location().to_string(); - MetadataLocation::new_with_table_location(warehouse_location).to_string() + warehouse_location } }; @@ -477,6 +477,8 @@ impl Catalog for S3TablesCatalog { let metadata = TableMetadataBuilder::from_table_creation(creation)? .build()? .metadata; + let metadata_location = + MetadataLocation::new_with_table(metadata_location, &metadata.properties).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; // update metadata location diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index 77b35a228f..9843bf6ca7 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -826,7 +826,8 @@ impl Catalog for SqlCatalog { .build()? .metadata; let tbl_metadata_location = - MetadataLocation::new_with_table_location(location.clone()).to_string(); + MetadataLocation::new_with_table(location.clone(), &tbl_metadata.properties) + .to_string(); tbl_metadata .write_to(&self.fileio, &tbl_metadata_location) diff --git a/crates/iceberg/src/catalog/memory/catalog.rs b/crates/iceberg/src/catalog/memory/catalog.rs index cfa3dc6b52..2551b5f2b7 100644 --- a/crates/iceberg/src/catalog/memory/catalog.rs +++ b/crates/iceberg/src/catalog/memory/catalog.rs @@ -275,7 +275,8 @@ impl Catalog for MemoryCatalog { let metadata = TableMetadataBuilder::from_table_creation(table_creation)? .build()? .metadata; - let metadata_location = MetadataLocation::new_with_table_location(location).to_string(); + let metadata_location = + MetadataLocation::new_with_table(location, &metadata.properties).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index dacf5c1dda..f46cfd19ac 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use std::fmt::Display; use std::str::FromStr; @@ -52,7 +53,10 @@ impl MetadataLocation { /// Creates a completely new metadata location starting at version 0, /// with compression settings from the table's properties. /// Only used for creating a new table. For updates, see `with_next_version`. - pub fn new_with_table(table_location: impl ToString, properties: &std::collections::HashMap) -> Self { + pub fn new_with_table( + table_location: impl ToString, + properties: &HashMap, + ) -> Self { let compression_suffix = properties .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) .and_then(|codec| match codec.to_lowercase().as_str() { @@ -91,12 +95,10 @@ impl MetadataLocation { /// Parses a file name of the format `-.metadata.json` /// or with compression: `-.gz.metadata.json`. fn parse_file_name(file_name: &str) -> Result<(i32, Uuid, Option)> { - let stripped = file_name - .strip_suffix(".metadata.json") - .ok_or(Error::new( - ErrorKind::Unexpected, - format!("Invalid metadata file ending: {file_name}"), - ))?; + let stripped = file_name.strip_suffix(".metadata.json").ok_or(Error::new( + ErrorKind::Unexpected, + format!("Invalid metadata file ending: {file_name}"), + ))?; // Check for compression suffix (e.g., .gz) let (stripped, compression_suffix) = if let Some(s) = stripped.strip_suffix(".gz") { @@ -105,14 +107,16 @@ impl MetadataLocation { (stripped, None) }; - let (version, id) = stripped - .split_once('-') - .ok_or(Error::new( - ErrorKind::Unexpected, - format!("Invalid metadata file name format: {file_name}"), - ))?; + let (version, id) = stripped.split_once('-').ok_or(Error::new( + ErrorKind::Unexpected, + format!("Invalid metadata file name format: {file_name}"), + ))?; - Ok((version.parse::()?, Uuid::parse_str(id)?, compression_suffix)) + Ok(( + version.parse::()?, + Uuid::parse_str(id)?, + compression_suffix, + )) } } @@ -317,7 +321,10 @@ mod test { assert_eq!(location.compression_suffix, Some(".gz".to_string())); assert_eq!( location.to_string(), - format!("/test/table/metadata/00000-{}.gz.metadata.json", location.id) + format!( + "/test/table/metadata/00000-{}.gz.metadata.json", + location.id + ) ); // Test with "none" codec (explicitly no compression) diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index aaa56408b7..037dbe1230 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -32,9 +32,12 @@ where ::Err: Display, { properties.get(key).map_or(Ok(default), |value| { - value - .parse::() - .map_err(|e| Error::new(ErrorKind::DataInvalid, format!("Invalid value for {key}: {e}"))) + value.parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Invalid value for {key}: {e}"), + ) + }) }) } From 4f033e87a569c0ecec1acd5bd63146d8289a542d Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 18:37:06 +0000 Subject: [PATCH 15/25] don't call deprecated method --- crates/catalog/glue/src/catalog.rs | 3 ++- crates/catalog/glue/src/utils.rs | 3 ++- crates/catalog/hms/src/catalog.rs | 3 ++- crates/catalog/hms/src/utils.rs | 2 +- crates/catalog/s3tables/src/catalog.rs | 3 ++- crates/catalog/sql/src/catalog.rs | 2 +- crates/iceberg/src/catalog/memory/catalog.rs | 2 +- crates/iceberg/src/catalog/metadata_location.rs | 14 +++++++------- 8 files changed, 18 insertions(+), 14 deletions(-) diff --git a/crates/catalog/glue/src/catalog.rs b/crates/catalog/glue/src/catalog.rs index 21a1ed1b28..66ad7dee07 100644 --- a/crates/catalog/glue/src/catalog.rs +++ b/crates/catalog/glue/src/catalog.rs @@ -471,7 +471,8 @@ impl Catalog for GlueCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_table(location.clone(), &metadata.properties).to_string(); + MetadataLocation::new_with_properties(location.clone(), &metadata.properties) + .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/glue/src/utils.rs b/crates/catalog/glue/src/utils.rs index 50777d7cf4..086acb1f1f 100644 --- a/crates/catalog/glue/src/utils.rs +++ b/crates/catalog/glue/src/utils.rs @@ -307,7 +307,8 @@ mod tests { let table_name = "my_table".to_string(); let location = "s3a://warehouse/hive".to_string(); let properties = HashMap::new(); - let metadata_location = MetadataLocation::new_with_table(location, &properties).to_string(); + let metadata_location = + MetadataLocation::new_with_properties(location, &properties).to_string(); let schema = Schema::builder() .with_schema_id(1) .with_fields(vec![ diff --git a/crates/catalog/hms/src/catalog.rs b/crates/catalog/hms/src/catalog.rs index 5c5d1b89b0..e3136c0304 100644 --- a/crates/catalog/hms/src/catalog.rs +++ b/crates/catalog/hms/src/catalog.rs @@ -443,7 +443,8 @@ impl Catalog for HmsCatalog { .metadata; let metadata_location = - MetadataLocation::new_with_table(location.clone(), &metadata.properties).to_string(); + MetadataLocation::new_with_properties(location.clone(), &metadata.properties) + .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/hms/src/utils.rs b/crates/catalog/hms/src/utils.rs index cee1881645..241a9c137b 100644 --- a/crates/catalog/hms/src/utils.rs +++ b/crates/catalog/hms/src/utils.rs @@ -345,7 +345,7 @@ mod tests { let location = "s3a://warehouse/hms".to_string(); let properties = HashMap::new(); let metadata_location = - MetadataLocation::new_with_table(location.clone(), &properties).to_string(); + MetadataLocation::new_with_properties(location.clone(), &properties).to_string(); let schema = Schema::builder() .with_schema_id(1) .with_fields(vec![ diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index 161257c2a7..7b822b40e2 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -478,7 +478,8 @@ impl Catalog for S3TablesCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_table(metadata_location, &metadata.properties).to_string(); + MetadataLocation::new_with_properties(metadata_location, &metadata.properties) + .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; // update metadata location diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index 9843bf6ca7..07ea14255e 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -826,7 +826,7 @@ impl Catalog for SqlCatalog { .build()? .metadata; let tbl_metadata_location = - MetadataLocation::new_with_table(location.clone(), &tbl_metadata.properties) + MetadataLocation::new_with_properties(location.clone(), &tbl_metadata.properties) .to_string(); tbl_metadata diff --git a/crates/iceberg/src/catalog/memory/catalog.rs b/crates/iceberg/src/catalog/memory/catalog.rs index 2551b5f2b7..e71f3833cf 100644 --- a/crates/iceberg/src/catalog/memory/catalog.rs +++ b/crates/iceberg/src/catalog/memory/catalog.rs @@ -276,7 +276,7 @@ impl Catalog for MemoryCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_table(location, &metadata.properties).to_string(); + MetadataLocation::new_with_properties(location, &metadata.properties).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index f46cfd19ac..16d83a36ca 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -39,7 +39,7 @@ impl MetadataLocation { /// Only used for creating a new table. For updates, see `with_next_version`. #[deprecated( since = "0.7.0", - note = "Use new_with_table instead to properly handle compression settings" + note = "Use new_with_properties instead to properly handle compression settings" )] pub fn new_with_table_location(table_location: impl ToString) -> Self { Self { @@ -53,7 +53,7 @@ impl MetadataLocation { /// Creates a completely new metadata location starting at version 0, /// with compression settings from the table's properties. /// Only used for creating a new table. For updates, see `with_next_version`. - pub fn new_with_table( + pub fn new_with_properties( table_location: impl ToString, properties: &HashMap, ) -> Self { @@ -297,12 +297,12 @@ mod test { } #[test] - fn test_metadata_location_new_with_table() { + fn test_metadata_location_new_with_properties() { use std::collections::HashMap; // Test with no compression let props_none = HashMap::new(); - let location = MetadataLocation::new_with_table("/test/table", &props_none); + let location = MetadataLocation::new_with_properties("/test/table", &props_none); assert_eq!(location.table_location, "/test/table"); assert_eq!(location.version, 0); assert_eq!(location.compression_suffix, None); @@ -317,7 +317,7 @@ mod test { "write.metadata.compression-codec".to_string(), "gzip".to_string(), ); - let location = MetadataLocation::new_with_table("/test/table", &props_gzip); + let location = MetadataLocation::new_with_properties("/test/table", &props_gzip); assert_eq!(location.compression_suffix, Some(".gz".to_string())); assert_eq!( location.to_string(), @@ -333,7 +333,7 @@ mod test { "write.metadata.compression-codec".to_string(), "none".to_string(), ); - let location = MetadataLocation::new_with_table("/test/table", &props_explicit_none); + let location = MetadataLocation::new_with_properties("/test/table", &props_explicit_none); assert_eq!(location.compression_suffix, None); // Test case insensitivity @@ -342,7 +342,7 @@ mod test { "write.metadata.compression-codec".to_string(), "GZIP".to_string(), ); - let location = MetadataLocation::new_with_table("/test/table", &props_gzip_upper); + let location = MetadataLocation::new_with_properties("/test/table", &props_gzip_upper); assert_eq!(location.compression_suffix, Some(".gz".to_string())); } } From 2db8b89529e66c281df1788bc71c32209ca34332 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 19:04:11 +0000 Subject: [PATCH 16/25] with_next_version --- .../iceberg/src/catalog/metadata_location.rs | 63 +++++++++++++++++++ crates/iceberg/src/catalog/mod.rs | 9 ++- 2 files changed, 69 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index 16d83a36ca..aaf9b858be 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -74,6 +74,7 @@ impl MetadataLocation { } /// Creates a new metadata location for an updated metadata file. + /// Preserves the compression settings from the current location. pub fn with_next_version(&self) -> Self { Self { table_location: self.table_location.clone(), @@ -83,6 +84,26 @@ impl MetadataLocation { } } + /// Creates a new metadata location for an updated metadata file. + /// Takes table properties to determine compression settings, which may have changed + /// from the previous version. + pub fn with_next_version_and_properties(&self, properties: &HashMap) -> Self { + let compression_suffix = properties + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .and_then(|codec| match codec.to_lowercase().as_str() { + "gzip" => Some(".gz".to_string()), + "none" | "" => None, + _ => None, + }); + + Self { + table_location: self.table_location.clone(), + version: self.version + 1, + id: Uuid::new_v4(), + compression_suffix, + } + } + fn parse_metadata_path_prefix(path: &str) -> Result { let prefix = path.strip_suffix("/metadata").ok_or(Error::new( ErrorKind::Unexpected, @@ -154,6 +175,7 @@ impl FromStr for MetadataLocation { #[cfg(test)] mod test { + use std::collections::HashMap; use std::str::FromStr; use uuid::Uuid; @@ -345,4 +367,45 @@ mod test { let location = MetadataLocation::new_with_properties("/test/table", &props_gzip_upper); assert_eq!(location.compression_suffix, Some(".gz".to_string())); } + + #[test] + fn test_with_next_version_and_properties() { + use std::collections::HashMap; + + // Start with a location without compression + let props_none = HashMap::new(); + let location = MetadataLocation::new_with_properties("/test/table", &props_none); + assert_eq!(location.compression_suffix, None); + assert_eq!(location.version, 0); + + // Update to next version with gzip compression + let mut props_gzip = HashMap::new(); + props_gzip.insert( + "write.metadata.compression-codec".to_string(), + "gzip".to_string(), + ); + let next_location = location.with_next_version_and_properties(&props_gzip); + assert_eq!(next_location.compression_suffix, Some(".gz".to_string())); + assert_eq!(next_location.version, 1); + assert_eq!( + next_location.to_string(), + format!( + "/test/table/metadata/00001-{}.gz.metadata.json", + next_location.id + ) + ); + + // Update to next version without compression (changed from gzip) + let props_none_again = HashMap::new(); + let final_location = next_location.with_next_version_and_properties(&props_none_again); + assert_eq!(final_location.compression_suffix, None); + assert_eq!(final_location.version, 2); + assert_eq!( + final_location.to_string(), + format!( + "/test/table/metadata/00002-{}.metadata.json", + final_location.id + ) + ); + } } diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 27d5edaedb..e9a15814de 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -353,13 +353,16 @@ impl TableCommit { metadata_builder = update.apply(metadata_builder)?; } - // Bump the version of metadata + // Build the new metadata + let new_metadata = metadata_builder.build()?.metadata; + + // Bump the version of metadata, using properties from the new metadata let new_metadata_location = MetadataLocation::from_str(current_metadata_location)? - .with_next_version() + .with_next_version_and_properties(&new_metadata.properties) .to_string(); Ok(table - .with_metadata(Arc::new(metadata_builder.build()?.metadata)) + .with_metadata(Arc::new(new_metadata)) .with_metadata_location(new_metadata_location)) } } From 8b63161987d6dc5abf0257446f3b9d4358752df3 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 19:40:24 +0000 Subject: [PATCH 17/25] refactor --- .../iceberg/src/catalog/metadata_location.rs | 39 +++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index aaf9b858be..bc3c601df5 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -35,10 +35,21 @@ pub struct MetadataLocation { } impl MetadataLocation { + /// Determines the compression suffix from table properties. + fn compression_suffix_from_properties(properties: &HashMap) -> Option { + properties + .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) + .and_then(|codec| match codec.to_lowercase().as_str() { + "gzip" => Some(".gz".to_string()), + "none" | "" => None, + _ => None, + }) + } + /// Creates a completely new metadata location starting at version 0. /// Only used for creating a new table. For updates, see `with_next_version`. #[deprecated( - since = "0.7.0", + since = "0.8.0", note = "Use new_with_properties instead to properly handle compression settings" )] pub fn new_with_table_location(table_location: impl ToString) -> Self { @@ -57,24 +68,20 @@ impl MetadataLocation { table_location: impl ToString, properties: &HashMap, ) -> Self { - let compression_suffix = properties - .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) - .and_then(|codec| match codec.to_lowercase().as_str() { - "gzip" => Some(".gz".to_string()), - "none" | "" => None, - _ => None, - }); - Self { table_location: table_location.to_string(), version: 0, id: Uuid::new_v4(), - compression_suffix, + compression_suffix: Self::compression_suffix_from_properties(properties), } } /// Creates a new metadata location for an updated metadata file. /// Preserves the compression settings from the current location. + #[deprecated( + since = "0.8.0", + note = "Use with_next_version_and_properties instead to properly handle compression settings changes" + )] pub fn with_next_version(&self) -> Self { Self { table_location: self.table_location.clone(), @@ -88,19 +95,11 @@ impl MetadataLocation { /// Takes table properties to determine compression settings, which may have changed /// from the previous version. pub fn with_next_version_and_properties(&self, properties: &HashMap) -> Self { - let compression_suffix = properties - .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) - .and_then(|codec| match codec.to_lowercase().as_str() { - "gzip" => Some(".gz".to_string()), - "none" | "" => None, - _ => None, - }); - Self { table_location: self.table_location.clone(), version: self.version + 1, id: Uuid::new_v4(), - compression_suffix, + compression_suffix: Self::compression_suffix_from_properties(properties), } } @@ -298,9 +297,9 @@ mod test { } #[test] + #[allow(deprecated)] fn test_metadata_location_with_next_version() { let test_cases = vec![ - #[allow(deprecated)] MetadataLocation::new_with_table_location("/abc"), MetadataLocation::from_str( "/abc/def/metadata/1234567-2cd22b57-5127-4198-92ba-e4e67c79821b.metadata.json", From 9308050f9c63b9757837b09ab3e0a6dfeacceb24 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 19:43:40 +0000 Subject: [PATCH 18/25] fix imports --- crates/iceberg/src/catalog/metadata_location.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index bc3c601df5..acaa92db18 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -319,8 +319,6 @@ mod test { #[test] fn test_metadata_location_new_with_properties() { - use std::collections::HashMap; - // Test with no compression let props_none = HashMap::new(); let location = MetadataLocation::new_with_properties("/test/table", &props_none); @@ -369,8 +367,6 @@ mod test { #[test] fn test_with_next_version_and_properties() { - use std::collections::HashMap; - // Start with a location without compression let props_none = HashMap::new(); let location = MetadataLocation::new_with_properties("/test/table", &props_none); From 9bd46d6f36e6dd038698d02ad42f7a7de4a644a1 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Thu, 20 Nov 2025 23:23:53 +0000 Subject: [PATCH 19/25] fix compile --- crates/catalog/glue/src/catalog.rs | 2 +- crates/catalog/hms/src/catalog.rs | 2 +- crates/catalog/s3tables/src/catalog.rs | 2 +- crates/catalog/sql/src/catalog.rs | 2 +- crates/iceberg/src/catalog/memory/catalog.rs | 2 +- crates/iceberg/src/catalog/mod.rs | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/catalog/glue/src/catalog.rs b/crates/catalog/glue/src/catalog.rs index 66ad7dee07..7e302db79e 100644 --- a/crates/catalog/glue/src/catalog.rs +++ b/crates/catalog/glue/src/catalog.rs @@ -471,7 +471,7 @@ impl Catalog for GlueCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_properties(location.clone(), &metadata.properties) + MetadataLocation::new_with_properties(location.clone(), metadata.properties()) .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/hms/src/catalog.rs b/crates/catalog/hms/src/catalog.rs index e3136c0304..59cef3bf17 100644 --- a/crates/catalog/hms/src/catalog.rs +++ b/crates/catalog/hms/src/catalog.rs @@ -443,7 +443,7 @@ impl Catalog for HmsCatalog { .metadata; let metadata_location = - MetadataLocation::new_with_properties(location.clone(), &metadata.properties) + MetadataLocation::new_with_properties(location.clone(), metadata.properties()) .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index 7b822b40e2..b7179f7e9a 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -478,7 +478,7 @@ impl Catalog for S3TablesCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_properties(metadata_location, &metadata.properties) + MetadataLocation::new_with_properties(metadata_location, metadata.properties()) .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index 07ea14255e..8006533a90 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -826,7 +826,7 @@ impl Catalog for SqlCatalog { .build()? .metadata; let tbl_metadata_location = - MetadataLocation::new_with_properties(location.clone(), &tbl_metadata.properties) + MetadataLocation::new_with_properties(location.clone(), tbl_metadata.properties()) .to_string(); tbl_metadata diff --git a/crates/iceberg/src/catalog/memory/catalog.rs b/crates/iceberg/src/catalog/memory/catalog.rs index e71f3833cf..91e95de8a7 100644 --- a/crates/iceberg/src/catalog/memory/catalog.rs +++ b/crates/iceberg/src/catalog/memory/catalog.rs @@ -276,7 +276,7 @@ impl Catalog for MemoryCatalog { .build()? .metadata; let metadata_location = - MetadataLocation::new_with_properties(location, &metadata.properties).to_string(); + MetadataLocation::new_with_properties(location, metadata.properties()).to_string(); metadata.write_to(&self.file_io, &metadata_location).await?; diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index e9a15814de..070a11dfb2 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -358,7 +358,7 @@ impl TableCommit { // Bump the version of metadata, using properties from the new metadata let new_metadata_location = MetadataLocation::from_str(current_metadata_location)? - .with_next_version_and_properties(&new_metadata.properties) + .with_next_version_and_properties(new_metadata.properties()) .to_string(); Ok(table From f3276cef372f3f91172b1e199230a96fac08e224 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Fri, 21 Nov 2025 00:39:47 +0000 Subject: [PATCH 20/25] fix clipp --- crates/catalog/s3tables/src/catalog.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index b7179f7e9a..958b34b144 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -467,8 +467,7 @@ impl Catalog for S3TablesCatalog { .send() .await .map_err(from_aws_sdk_error)?; - let warehouse_location = get_resp.warehouse_location().to_string(); - warehouse_location + get_resp.warehouse_location().to_string(); } }; From 27cbc0b8e235c6175613901ea1ee01b64c93251f Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Fri, 21 Nov 2025 01:11:59 +0000 Subject: [PATCH 21/25] remove ; --- crates/catalog/s3tables/src/catalog.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index 958b34b144..c9464ee618 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -467,7 +467,7 @@ impl Catalog for S3TablesCatalog { .send() .await .map_err(from_aws_sdk_error)?; - get_resp.warehouse_location().to_string(); + get_resp.warehouse_location().to_string() } }; From bdc6bb59a13107c2e5ff95a76c2d802ff44eba00 Mon Sep 17 00:00:00 2001 From: emkornfield Date: Fri, 21 Nov 2025 08:54:12 -0800 Subject: [PATCH 22/25] Update crates/iceberg/src/spec/table_properties.rs Co-authored-by: Kevin Liu --- crates/iceberg/src/spec/table_properties.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index 037dbe1230..df40a1ed10 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -191,7 +191,7 @@ impl TryFrom<&HashMap> for TableProperties { )?, metadata_compression_codec: props .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) - .and_then(|v| match v.as_str() { + .and_then(|v| match v.to_lowercase().as_str() { "none" | "" => None, codec => Some(codec.to_string()), }), From 05fdd81ec74bb50d0adf25ab7813b2d4537f9514 Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Fri, 21 Nov 2025 18:54:52 +0000 Subject: [PATCH 23/25] address comment on .gz --- crates/iceberg/src/catalog/metadata_location.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/catalog/metadata_location.rs b/crates/iceberg/src/catalog/metadata_location.rs index acaa92db18..29386944eb 100644 --- a/crates/iceberg/src/catalog/metadata_location.rs +++ b/crates/iceberg/src/catalog/metadata_location.rs @@ -24,6 +24,9 @@ use uuid::Uuid; use crate::spec::TableProperties; use crate::{Error, ErrorKind, Result}; +/// The file extension suffix for gzip compressed metadata files +const GZIP_SUFFIX: &str = ".gz"; + /// Helper for parsing a location of the format: `/metadata/-.metadata.json` /// or with compression: `/metadata/-.gz.metadata.json` #[derive(Clone, Debug, PartialEq)] @@ -40,7 +43,7 @@ impl MetadataLocation { properties .get(TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC) .and_then(|codec| match codec.to_lowercase().as_str() { - "gzip" => Some(".gz".to_string()), + "gzip" => Some(GZIP_SUFFIX.to_string()), "none" | "" => None, _ => None, }) @@ -121,8 +124,8 @@ impl MetadataLocation { ))?; // Check for compression suffix (e.g., .gz) - let (stripped, compression_suffix) = if let Some(s) = stripped.strip_suffix(".gz") { - (s, Some(".gz".to_string())) + let (stripped, compression_suffix) = if let Some(s) = stripped.strip_suffix(GZIP_SUFFIX) { + (s, Some(GZIP_SUFFIX.to_string())) } else { (stripped, None) }; From 207c0bf60862da5052cfb1c5204dca498742b67e Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Fri, 21 Nov 2025 18:57:25 +0000 Subject: [PATCH 24/25] add tests --- crates/iceberg/src/spec/table_properties.rs | 33 +++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/crates/iceberg/src/spec/table_properties.rs b/crates/iceberg/src/spec/table_properties.rs index df40a1ed10..c9258f2076 100644 --- a/crates/iceberg/src/spec/table_properties.rs +++ b/crates/iceberg/src/spec/table_properties.rs @@ -254,6 +254,39 @@ mod tests { assert_eq!(table_properties.metadata_compression_codec, None); } + #[test] + fn test_table_properties_compression_case_insensitive() { + // Test uppercase + let props_upper = HashMap::from([( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "GZIP".to_string(), + )]); + let table_properties = TableProperties::try_from(&props_upper).unwrap(); + assert_eq!( + table_properties.metadata_compression_codec, + Some("gzip".to_string()) + ); + + // Test mixed case + let props_mixed = HashMap::from([( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "GzIp".to_string(), + )]); + let table_properties = TableProperties::try_from(&props_mixed).unwrap(); + assert_eq!( + table_properties.metadata_compression_codec, + Some("gzip".to_string()) + ); + + // Test "NONE" should also be case-insensitive + let props_none_upper = HashMap::from([( + TableProperties::PROPERTY_METADATA_COMPRESSION_CODEC.to_string(), + "NONE".to_string(), + )]); + let table_properties = TableProperties::try_from(&props_none_upper).unwrap(); + assert_eq!(table_properties.metadata_compression_codec, None); + } + #[test] fn test_table_properties_valid() { let props = HashMap::from([ From e83c44978ae62da449521f6b9d68a2b93d81eb1b Mon Sep 17 00:00:00 2001 From: Micah Kornfield Date: Fri, 21 Nov 2025 19:14:04 +0000 Subject: [PATCH 25/25] fixes --- crates/catalog/s3tables/src/catalog.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index c9464ee618..815997f7b0 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -450,7 +450,7 @@ impl Catalog for S3TablesCatalog { // prepare metadata location. the warehouse location is generated by s3tables catalog, // which looks like: s3://e6c9bf20-991a-46fb-kni5xs1q2yxi3xxdyxzjzigdeop1quse2b--table-s3 - let metadata_location = match &creation.location { + let table_location = match &creation.location { Some(_) => { return Err(Error::new( ErrorKind::DataInvalid, @@ -472,12 +472,12 @@ impl Catalog for S3TablesCatalog { }; // write metadata to file - creation.location = Some(metadata_location.clone()); + creation.location = Some(table_location.clone()); let metadata = TableMetadataBuilder::from_table_creation(creation)? .build()? .metadata; let metadata_location = - MetadataLocation::new_with_properties(metadata_location, metadata.properties()) + MetadataLocation::new_with_properties(table_location, metadata.properties()) .to_string(); metadata.write_to(&self.file_io, &metadata_location).await?;