diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index c82f353fe..3bad52745 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -1207,6 +1207,12 @@ impl ManifestEntry { pub fn data_file(&self) -> &DataFile { &self.data_file } + + /// File sequence number indicating when the file was added. Inherited when null and status is 1 (added). + #[inline] + pub fn file_sequence_number(&self) -> Option { + self.file_sequence_number + } } /// Used to track additions and deletions in ManifestEntry. diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs index 15d5c99a1..6cb595b6c 100644 --- a/crates/iceberg/src/transaction.rs +++ b/crates/iceberg/src/transaction.rs @@ -18,29 +18,42 @@ //! This module contains transaction api. use std::cmp::Ordering; -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::future::Future; use std::mem::discriminant; use std::ops::RangeFrom; +use std::pin::Pin; use arrow_array::StringArray; use futures::TryStreamExt; use uuid::Uuid; use crate::error::Result; -use crate::io::OutputFile; +use crate::io::FileIO; use crate::spec::{ - DataFile, DataFileFormat, FormatVersion, ManifestEntry, ManifestFile, ManifestListWriter, - ManifestWriterBuilder, NullOrder, Operation, Snapshot, SnapshotReference, SnapshotRetention, - SortDirection, SortField, SortOrder, Struct, StructType, Summary, Transform, MAIN_BRANCH, + DataFile, DataFileFormat, FormatVersion, ManifestContentType, ManifestEntry, ManifestFile, + ManifestListWriter, ManifestStatus, ManifestWriter, ManifestWriterBuilder, NullOrder, + Operation, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, + Struct, StructType, Summary, Transform, MAIN_BRANCH, }; use crate::table::Table; +use crate::utils::bin::ListPacker; use crate::writer::file_writer::ParquetWriter; use crate::TableUpdate::UpgradeFormatVersion; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; const META_ROOT_PATH: &str = "metadata"; +/// Target size of manifest file when merging manifests. +pub const MANIFEST_TARGET_SIZE_BYTES: &str = "commit.manifest.target-size-bytes"; +const MANIFEST_TARGET_SIZE_BYTES_DEFAULT: u32 = 8 * 1024 * 1024; // 8 MB +/// Minimum number of manifests to merge. +pub const MANIFEST_MIN_MERGE_COUNT: &str = "commit.manifest.min-count-to-merge"; +const MANIFEST_MIN_MERGE_COUNT_DEFAULT: u32 = 100; +/// Whether allow to merge manifests. +pub const MANIFEST_MERGE_ENABLED: &str = "commit.manifest-merge.enabled"; +const MANIFEST_MERGE_ENABLED_DEFAULT: bool = false; + /// Table transaction. pub struct Transaction<'a> { table: &'a Table, @@ -148,6 +161,22 @@ impl<'a> Transaction<'a> { ) } + /// Creates a merge append action. + pub fn merge_append( + self, + commit_uuid: Option, + key_metadata: Vec, + ) -> Result> { + let snapshot_id = self.generate_unique_snapshot_id(); + MergeAppendAction::new( + self, + snapshot_id, + commit_uuid.unwrap_or_else(Uuid::now_v7), + key_metadata, + HashMap::new(), + ) + } + /// Creates replace sort order action. pub fn replace_sort_order(self) -> ReplaceSortOrderAction<'a> { ReplaceSortOrderAction { @@ -305,6 +334,86 @@ impl<'a> FastAppendAction<'a> { } } +/// MergeAppendAction is a transaction action similar to fast append except that it will merge manifests +/// based on the target size. +pub struct MergeAppendAction<'a> { + snapshot_produce_action: SnapshotProduceAction<'a>, + target_size_bytes: u32, + min_count_to_merge: u32, + merge_enabled: bool, +} + +impl<'a> MergeAppendAction<'a> { + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + tx: Transaction<'a>, + snapshot_id: i64, + commit_uuid: Uuid, + key_metadata: Vec, + snapshot_properties: HashMap, + ) -> Result { + let target_size_bytes: u32 = tx + .table + .metadata() + .properties() + .get(MANIFEST_TARGET_SIZE_BYTES) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + let min_count_to_merge: u32 = tx + .table + .metadata() + .properties() + .get(MANIFEST_MIN_MERGE_COUNT) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MIN_MERGE_COUNT_DEFAULT); + let merge_enabled = tx + .table + .metadata() + .properties() + .get(MANIFEST_MERGE_ENABLED) + .and_then(|s| s.parse().ok()) + .unwrap_or(MANIFEST_MERGE_ENABLED_DEFAULT); + Ok(Self { + snapshot_produce_action: SnapshotProduceAction::new( + tx, + snapshot_id, + key_metadata, + commit_uuid, + snapshot_properties, + )?, + target_size_bytes, + min_count_to_merge, + merge_enabled, + }) + } + + /// Add data files to the snapshot. + pub fn add_data_files( + &mut self, + data_files: impl IntoIterator, + ) -> Result<&mut Self> { + self.snapshot_produce_action.add_data_files(data_files)?; + Ok(self) + } + + /// Finished building the action and apply it to the transaction. + pub async fn apply(self) -> Result> { + if self.merge_enabled { + let process = MergeManifestProcess { + target_size_bytes: self.target_size_bytes, + min_count_to_merge: self.min_count_to_merge, + }; + self.snapshot_produce_action + .apply(FastAppendOperation, process) + .await + } else { + self.snapshot_produce_action + .apply(FastAppendOperation, DefaultManifestProcess) + .await + } + } +} + struct FastAppendOperation; impl SnapshotProduceOperation for FastAppendOperation { @@ -359,13 +468,208 @@ trait SnapshotProduceOperation: Send + Sync { struct DefaultManifestProcess; impl ManifestProcess for DefaultManifestProcess { - fn process_manifeset(&self, manifests: Vec) -> Vec { - manifests + async fn process_manifest<'a>( + &self, + _snapshot_producer: &mut SnapshotProduceAction<'a>, + manifests: Vec, + ) -> Result> { + Ok(manifests) + } +} + +struct MergeManifestManager { + target_size_bytes: u32, + min_count_to_merge: u32, + content: ManifestContentType, +} + +impl MergeManifestManager { + pub fn new( + target_size_bytes: u32, + min_count_to_merge: u32, + content: ManifestContentType, + ) -> Self { + Self { + target_size_bytes, + min_count_to_merge, + content, + } + } + + fn group_by_spec(&self, manifests: Vec) -> BTreeMap> { + let mut grouped_manifests = BTreeMap::new(); + for manifest in manifests { + grouped_manifests + .entry(manifest.partition_spec_id) + .or_insert_with(Vec::new) + .push(manifest); + } + grouped_manifests + } + + async fn merge_bin( + &self, + snapshot_id: i64, + file_io: FileIO, + manifest_bin: Vec, + mut writer: ManifestWriter, + ) -> Result { + for manifest_file in manifest_bin { + let manifest_file = manifest_file.load_manifest(&file_io).await?; + for manifest_entry in manifest_file.entries() { + if manifest_entry.status() == ManifestStatus::Deleted + && manifest_entry + .snapshot_id() + .is_some_and(|id| id == snapshot_id) + { + //only files deleted by this snapshot should be added to the new manifest + writer.add_delete_entry(manifest_entry.as_ref().clone())?; + } else if manifest_entry.status() == ManifestStatus::Added + && manifest_entry + .snapshot_id() + .is_some_and(|id| id == snapshot_id) + { + //added entries from this snapshot are still added, otherwise they should be existing + writer.add_entry(manifest_entry.as_ref().clone())?; + } else if manifest_entry.status() != ManifestStatus::Deleted { + // add all non-deleted files from the old manifest as existing files + writer.add_existing_entry(manifest_entry.as_ref().clone())?; + } + } + } + + writer.write_manifest_file().await + } + + async fn merge_group<'a>( + &self, + snapshot_produce: &mut SnapshotProduceAction<'a>, + first_manifest: &ManifestFile, + group_manifests: Vec, + ) -> Result> { + let packer: ListPacker = ListPacker::new(self.target_size_bytes); + let manifest_bins = + packer.pack(group_manifests, |manifest| manifest.manifest_length as u32); + + let manifest_merge_futures = manifest_bins + .into_iter() + .map(|manifest_bin| { + if manifest_bin.len() == 1 { + Ok(Box::pin(async { Ok(manifest_bin) }) + as Pin< + Box>> + Send>, + >) + } + // if the bin has the first manifest (the new data files or an appended manifest file) then only + // merge it if the number of manifests is above the minimum count. this is applied only to bins + // with an in-memory manifest so that large manifests don't prevent merging older groups. + else if manifest_bin + .iter() + .any(|manifest| manifest == first_manifest) + && manifest_bin.len() < self.min_count_to_merge as usize + { + Ok(Box::pin(async { Ok(manifest_bin) }) + as Pin< + Box>> + Send>, + >) + } else { + let writer = snapshot_produce.new_manifest_writer(&self.content)?; + let snapshot_id = snapshot_produce.snapshot_id; + let file_io = snapshot_produce.tx.table.file_io().clone(); + Ok((Box::pin(async move { + Ok(vec![ + self.merge_bin( + snapshot_id, + file_io, + manifest_bin, + writer, + ) + .await?, + ]) + })) + as Pin>> + Send>>) + } + }) + .collect::>> + Send>>>>>()?; + + let merged_bins: Vec> = + futures::future::join_all(manifest_merge_futures.into_iter()) + .await + .into_iter() + .collect::>>()?; + + Ok(merged_bins.into_iter().flatten().collect()) + } + + async fn merge_manifest<'a>( + &self, + snapshot_produce: &mut SnapshotProduceAction<'a>, + manifests: Vec, + ) -> Result> { + if manifests.is_empty() { + return Ok(manifests); + } + + let first_manifest = manifests[0].clone(); + + let group_manifests = self.group_by_spec(manifests); + + let mut merge_manifests = vec![]; + for (_spec_id, manifests) in group_manifests.into_iter().rev() { + merge_manifests.extend( + self.merge_group(snapshot_produce, &first_manifest, manifests) + .await?, + ); + } + + Ok(merge_manifests) + } +} + +struct MergeManifestProcess { + target_size_bytes: u32, + min_count_to_merge: u32, +} + +impl MergeManifestProcess { + pub fn new(target_size_bytes: u32, min_count_to_merge: u32) -> Self { + Self { + target_size_bytes, + min_count_to_merge, + } + } +} + +impl ManifestProcess for MergeManifestProcess { + async fn process_manifest<'a>( + &self, + snapshot_produce: &mut SnapshotProduceAction<'a>, + manifests: Vec, + ) -> Result> { + let (unmerg_data_manifests, unmerge_delete_manifest) = manifests + .into_iter() + .partition(|m| m.content == ManifestContentType::Data); + let mut data_manifests = { + let merge_manifest_manager = MergeManifestManager::new( + self.target_size_bytes, + self.min_count_to_merge, + ManifestContentType::Data, + ); + merge_manifest_manager + .merge_manifest(snapshot_produce, unmerg_data_manifests) + .await? + }; + data_manifests.extend(unmerge_delete_manifest); + Ok(data_manifests) } } trait ManifestProcess: Send + Sync { - fn process_manifeset(&self, manifests: Vec) -> Vec; + fn process_manifest<'a>( + &self, + snapshot_produce: &mut SnapshotProduceAction<'a>, + manifests: Vec, + ) -> impl Future>> + Send; } struct SnapshotProduceAction<'a> { @@ -455,7 +759,7 @@ impl<'a> SnapshotProduceAction<'a> { Ok(self) } - fn new_manifest_output(&mut self) -> Result { + fn new_manifest_writer(&mut self, content: &ManifestContentType) -> Result { let new_manifest_path = format!( "{}/{}/{}-m{}.{}", self.tx.table.metadata().location(), @@ -464,7 +768,27 @@ impl<'a> SnapshotProduceAction<'a> { self.manifest_counter.next().unwrap(), DataFileFormat::Avro ); - self.tx.table.file_io().new_output(new_manifest_path) + let output_file = self.tx.table.file_io().new_output(new_manifest_path)?; + let builder = ManifestWriterBuilder::new( + output_file, + Some(self.snapshot_id), + self.key_metadata.clone(), + self.tx.table.metadata().current_schema().clone(), + self.tx + .table + .metadata() + .default_partition_spec() + .as_ref() + .clone(), + ); + if self.tx.table.metadata().format_version() == FormatVersion::V1 { + Ok(builder.build_v1()) + } else { + match content { + ManifestContentType::Data => Ok(builder.build_v2_data()), + ManifestContentType::Deletes => Ok(builder.build_v2_deletes()), + } + } } // Write manifest file for added data files and return the ManifestFile for ManifestList. @@ -483,25 +807,9 @@ impl<'a> SnapshotProduceAction<'a> { builder.build() } }); - let mut writer = { - let builder = ManifestWriterBuilder::new( - self.new_manifest_output()?, - Some(self.snapshot_id), - self.key_metadata.clone(), - self.tx.table.metadata().current_schema().clone(), - self.tx - .table - .metadata() - .default_partition_spec() - .as_ref() - .clone(), - ); - if self.tx.table.metadata().format_version() == FormatVersion::V1 { - builder.build_v1() - } else { - builder.build_v2_data() - } - }; + // # TODO + // Fix this when we support append delete type data file + let mut writer = self.new_manifest_writer(&ManifestContentType::Data)?; for entry in manifest_entries { writer.add_entry(entry)?; } @@ -520,8 +828,9 @@ impl<'a> SnapshotProduceAction<'a> { let mut manifest_files = vec![added_manifest]; manifest_files.extend(existing_manifests); - let manifest_files = manifest_process.process_manifeset(manifest_files); - Ok(manifest_files) + manifest_process + .process_manifest(self, manifest_files) + .await } // # TODO diff --git a/crates/iceberg/src/utils.rs b/crates/iceberg/src/utils.rs index 00d3e69bd..1a60c82a7 100644 --- a/crates/iceberg/src/utils.rs +++ b/crates/iceberg/src/utils.rs @@ -40,3 +40,148 @@ pub(crate) fn available_parallelism() -> NonZeroUsize { NonZeroUsize::new(DEFAULT_PARALLELISM).unwrap() }) } + +pub mod bin { + use std::iter::Iterator; + use std::marker::PhantomData; + + use itertools::Itertools; + + struct Bin { + bin_weight: u32, + target_weight: u32, + items: Vec, + } + + impl Bin { + pub fn new(target_weight: u32) -> Self { + Bin { + bin_weight: 0, + target_weight, + items: Vec::new(), + } + } + + pub fn can_add(&self, weight: u32) -> bool { + self.bin_weight + weight <= self.target_weight + } + + pub fn add(&mut self, item: T, weight: u32) { + self.bin_weight += weight; + self.items.push(item); + } + + pub fn into_vec(self) -> Vec { + self.items + } + } + + /// ListPacker help to pack item into bin of item. Each bin has close to + /// target_weight. + pub(crate) struct ListPacker { + target_weight: u32, + _marker: PhantomData, + } + + impl ListPacker { + pub fn new(target_weight: u32) -> Self { + ListPacker { + target_weight, + _marker: PhantomData, + } + } + + pub fn pack(&self, items: Vec, weight_func: F) -> Vec> + where F: Fn(&T) -> u32 { + let mut bins: Vec> = vec![]; + for item in items { + let cur_weight = weight_func(&item); + let addable_bin = + if let Some(bin) = bins.iter_mut().find(|bin| bin.can_add(cur_weight)) { + bin + } else { + bins.push(Bin::new(self.target_weight)); + bins.last_mut().unwrap() + }; + addable_bin.add(item, cur_weight); + } + + bins.into_iter().map(|bin| bin.into_vec()).collect_vec() + } + } + + #[cfg(test)] + mod tests { + use super::*; + + #[test] + fn test_list_packer_basic_packing() { + let packer = ListPacker::new(10); + let items = vec![3, 4, 5, 6, 2, 1]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 3); + assert!(packed[0].iter().sum::() == 10); + assert!(packed[1].iter().sum::() == 5); + assert!(packed[2].iter().sum::() == 6); + } + + #[test] + fn test_list_packer_with_complex_items() { + #[derive(Debug, PartialEq)] + struct Item { + name: String, + size: u32, + } + + let packer = ListPacker::new(15); + let items = vec![ + Item { + name: "A".to_string(), + size: 7, + }, + Item { + name: "B".to_string(), + size: 8, + }, + Item { + name: "C".to_string(), + size: 5, + }, + Item { + name: "D".to_string(), + size: 6, + }, + ]; + + let packed = packer.pack(items, |item| item.size); + + assert_eq!(packed.len(), 2); + assert!(packed[0].iter().map(|x| x.size).sum::() <= 15); + assert!(packed[1].iter().map(|x| x.size).sum::() <= 15); + } + + #[test] + fn test_list_packer_single_large_item() { + let packer = ListPacker::new(10); + let items = vec![15, 5, 3]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 2); + assert!(packed[0].contains(&15)); + assert!(packed[1].iter().sum::() <= 10); + } + + #[test] + fn test_list_packer_empty_input() { + let packer = ListPacker::new(10); + let items: Vec = vec![]; + + let packed = packer.pack(items, |&x| x); + + assert_eq!(packed.len(), 0); + } + } +} diff --git a/crates/integration_tests/tests/shared_tests/merge_append_test.rs b/crates/integration_tests/tests/shared_tests/merge_append_test.rs new file mode 100644 index 000000000..32b655b20 --- /dev/null +++ b/crates/integration_tests/tests/shared_tests/merge_append_test.rs @@ -0,0 +1,206 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Integration tests for rest catalog. + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use iceberg::spec::{ + DataFile, ManifestEntry, ManifestStatus, NestedField, PrimitiveType, Schema, Type, +}; +use iceberg::table::Table; +use iceberg::transaction::{ + Transaction, MANIFEST_MERGE_ENABLED, MANIFEST_MIN_MERGE_COUNT, MANIFEST_TARGET_SIZE_BYTES, +}; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, TableCreation}; +use iceberg_catalog_rest::RestCatalog; +use parquet::file::properties::WriterProperties; + +use crate::get_shared_containers; +use crate::shared_tests::random_ns; + +async fn write_new_data_file(table: &Table) -> Vec { + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None); + let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); + let col1 = StringArray::from(vec![Some("foo"); 100]); + let col2 = Int32Array::from(vec![Some(1); 100]); + let col3 = BooleanArray::from(vec![Some(true); 100]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + data_file_writer.close().await.unwrap() +} + +#[tokio::test] +async fn test_append_data_file() { + let fixture = get_shared_containers(); + let rest_catalog = RestCatalog::new(fixture.catalog_config.clone()); + let ns = random_ns().await; + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + let mut table = rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Enable merge append for table + let tx = Transaction::new(&table); + table = tx + .set_properties(HashMap::from([ + (MANIFEST_MERGE_ENABLED.to_string(), "true".to_string()), + (MANIFEST_MIN_MERGE_COUNT.to_string(), "4".to_string()), + (MANIFEST_TARGET_SIZE_BYTES.to_string(), "7000".to_string()), + ])) + .unwrap() + .commit(&rest_catalog) + .await + .unwrap(); + + // fast append data file 3 time to create 3 manifest + let mut original_manifest_entries = vec![]; + for _ in 0..3 { + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + append_action.add_data_files(data_file.clone()).unwrap(); + let tx = append_action.apply().await.unwrap(); + table = tx.commit(&rest_catalog).await.unwrap() + } + let manifest_list = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list.entries().len(), 3); + + // construct test data + for (idx, entry) in manifest_list.entries().iter().enumerate() { + let manifest = entry.load_manifest(table.file_io()).await.unwrap(); + assert!(manifest.entries().len() == 1); + + // For this first manifest, it will be pack with the first additional manifest and + // the count(2) is less than the min merge count(4), so these two will not merge. + // See detail: `MergeManifestProcess::merge_group` + if idx == 0 { + original_manifest_entries.push(Arc::new( + ManifestEntry::builder() + .status(ManifestStatus::Added) + .snapshot_id(manifest.entries()[0].snapshot_id().unwrap()) + .sequence_number(manifest.entries()[0].sequence_number().unwrap()) + .file_sequence_number(manifest.entries()[0].file_sequence_number().unwrap()) + .data_file(manifest.entries()[0].data_file().clone()) + .build(), + )); + } else { + original_manifest_entries.push(Arc::new( + ManifestEntry::builder() + .status(ManifestStatus::Existing) + .snapshot_id(manifest.entries()[0].snapshot_id().unwrap()) + .sequence_number(manifest.entries()[0].sequence_number().unwrap()) + .file_sequence_number(manifest.entries()[0].file_sequence_number().unwrap()) + .data_file(manifest.entries()[0].data_file().clone()) + .build(), + )); + } + } + + // append data file with merge append, 4 data file will be merged to two manifest + let data_file = write_new_data_file(&table).await; + let tx = Transaction::new(&table); + let mut merge_append_action = tx.merge_append(None, vec![]).unwrap(); + merge_append_action + .add_data_files(data_file.clone()) + .unwrap(); + let tx = merge_append_action.apply().await.unwrap(); + table = tx.commit(&rest_catalog).await.unwrap(); + // Check manifest file + let manifest_list = table + .metadata() + .current_snapshot() + .unwrap() + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(manifest_list.entries().len(), 3); + { + let manifest = manifest_list.entries()[1] + .load_manifest(table.file_io()) + .await + .unwrap(); + assert!(manifest.entries().len() == 1); + original_manifest_entries.retain(|entry| !manifest.entries().contains(entry)); + assert!(original_manifest_entries.len() == 2); + } + { + let manifest = manifest_list.entries()[2] + .load_manifest(table.file_io()) + .await + .unwrap(); + assert!(manifest.entries().len() == 2); + for original_entry in original_manifest_entries.iter() { + assert!(manifest.entries().contains(original_entry)); + } + } +} diff --git a/crates/integration_tests/tests/shared_tests/mod.rs b/crates/integration_tests/tests/shared_tests/mod.rs index feb1c4e58..e1836e859 100644 --- a/crates/integration_tests/tests/shared_tests/mod.rs +++ b/crates/integration_tests/tests/shared_tests/mod.rs @@ -27,6 +27,7 @@ mod append_data_file_test; mod append_partition_data_file_test; mod conflict_commit_test; mod datafusion; +mod merge_append_test; mod read_evolved_schema; mod read_positional_deletes; mod scan_all_type;