iceberg/writer/partitioning/
clustered_writer.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! This module provides the `ClusteredWriter` implementation.
19
20use std::collections::HashSet;
21use std::marker::PhantomData;
22
23use async_trait::async_trait;
24
25use crate::spec::{PartitionKey, Struct};
26use crate::writer::partitioning::PartitioningWriter;
27use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder};
28use crate::{Error, ErrorKind, Result};
29
30/// A writer that writes data to a single partition at a time.
31///
32/// This writer expects input data to be sorted by partition key. It maintains only one
33/// active writer at a time, making it memory efficient for sorted data.
34///
35/// # Type Parameters
36///
37/// * `B` - The inner writer builder type
38/// * `I` - Input type (defaults to `RecordBatch`)
39/// * `O` - Output collection type (defaults to `Vec<DataFile>`)
40pub struct ClusteredWriter<B, I = DefaultInput, O = DefaultOutput>
41where
42    B: IcebergWriterBuilder<I, O>,
43    O: IntoIterator + FromIterator<<O as IntoIterator>::Item>,
44    <O as IntoIterator>::Item: Clone,
45{
46    inner_builder: B,
47    current_writer: Option<B::R>,
48    current_partition: Option<Struct>,
49    closed_partitions: HashSet<Struct>,
50    output: Vec<<O as IntoIterator>::Item>,
51    _phantom: PhantomData<I>,
52}
53
54impl<B, I, O> ClusteredWriter<B, I, O>
55where
56    B: IcebergWriterBuilder<I, O>,
57    I: Send + 'static,
58    O: IntoIterator + FromIterator<<O as IntoIterator>::Item>,
59    <O as IntoIterator>::Item: Send + Clone,
60{
61    /// Create a new `ClusteredWriter`.
62    pub fn new(inner_builder: B) -> Self {
63        Self {
64            inner_builder,
65            current_writer: None,
66            current_partition: None,
67            closed_partitions: HashSet::new(),
68            output: Vec::new(),
69            _phantom: PhantomData,
70        }
71    }
72
73    /// Closes the current writer if it exists, flushes the written data to output, and record closed partition.
74    async fn close_current_writer(&mut self) -> Result<()> {
75        if let Some(mut writer) = self.current_writer.take() {
76            self.output.extend(writer.close().await?);
77
78            // Add the current partition to the set of closed partitions
79            if let Some(current_partition) = self.current_partition.take() {
80                self.closed_partitions.insert(current_partition);
81            }
82        }
83
84        Ok(())
85    }
86}
87
88#[async_trait]
89impl<B, I, O> PartitioningWriter<I, O> for ClusteredWriter<B, I, O>
90where
91    B: IcebergWriterBuilder<I, O>,
92    I: Send + 'static,
93    O: IntoIterator + FromIterator<<O as IntoIterator>::Item> + Send + 'static,
94    <O as IntoIterator>::Item: Send + Clone,
95{
96    async fn write(&mut self, partition_key: PartitionKey, input: I) -> Result<()> {
97        let partition_value = partition_key.data();
98
99        // Check if this partition has been closed already
100        if self.closed_partitions.contains(partition_value) {
101            return Err(Error::new(
102                ErrorKind::Unexpected,
103                format!(
104                    "The input is not sorted! Cannot write to partition that was previously closed: {partition_key:?}"
105                ),
106            ));
107        }
108
109        // Check if we need to switch to a new partition
110        let need_new_writer = match &self.current_partition {
111            Some(current) => current != partition_value,
112            None => true,
113        };
114
115        if need_new_writer {
116            self.close_current_writer().await?;
117
118            // Create a new writer for the new partition
119            self.current_writer = Some(
120                self.inner_builder
121                    .build(Some(partition_key.clone()))
122                    .await?,
123            );
124            self.current_partition = Some(partition_value.clone());
125        }
126
127        // do write
128        self.current_writer
129            .as_mut()
130            .expect("Writer should be initialized")
131            .write(input)
132            .await
133    }
134
135    async fn close(mut self) -> Result<O> {
136        self.close_current_writer().await?;
137
138        // Collect all output items into the output collection type
139        Ok(O::from_iter(self.output))
140    }
141}
142
143#[cfg(test)]
144mod tests {
145    use std::collections::HashMap;
146    use std::sync::Arc;
147
148    use arrow_array::{Int32Array, RecordBatch, StringArray};
149    use arrow_schema::{DataType, Field, Schema};
150    use parquet::arrow::PARQUET_FIELD_ID_META_KEY;
151    use parquet::file::properties::WriterProperties;
152    use tempfile::TempDir;
153
154    use super::*;
155    use crate::io::FileIOBuilder;
156    use crate::spec::{DataFileFormat, NestedField, PrimitiveType, Type};
157    use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder;
158    use crate::writer::file_writer::ParquetWriterBuilder;
159    use crate::writer::file_writer::location_generator::{
160        DefaultFileNameGenerator, DefaultLocationGenerator,
161    };
162    use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder;
163
164    #[tokio::test]
165    async fn test_clustered_writer_single_partition() -> Result<()> {
166        let temp_dir = TempDir::new()?;
167        let file_io = FileIOBuilder::new_fs_io().build()?;
168        let location_gen = DefaultLocationGenerator::with_data_location(
169            temp_dir.path().to_str().unwrap().to_string(),
170        );
171        let file_name_gen =
172            DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet);
173
174        // Create schema with partition field
175        let schema = Arc::new(
176            crate::spec::Schema::builder()
177                .with_schema_id(1)
178                .with_fields(vec![
179                    NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(),
180                    NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(),
181                    NestedField::required(3, "region", Type::Primitive(PrimitiveType::String))
182                        .into(),
183                ])
184                .build()?,
185        );
186
187        // Create partition spec and key
188        let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?;
189        let partition_value =
190            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]);
191        let partition_key =
192            crate::spec::PartitionKey::new(partition_spec, schema.clone(), partition_value.clone());
193
194        // Create writer builder
195        let parquet_writer_builder =
196            ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone());
197
198        // Create rolling file writer builder
199        let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size(
200            parquet_writer_builder,
201            file_io.clone(),
202            location_gen,
203            file_name_gen,
204        );
205
206        // Create data file writer builder
207        let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder);
208
209        // Create clustered writer
210        let mut writer = ClusteredWriter::new(data_file_writer_builder);
211
212        // Create test data with proper field ID metadata
213        let arrow_schema = Schema::new(vec![
214            Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([(
215                PARQUET_FIELD_ID_META_KEY.to_string(),
216                1.to_string(),
217            )])),
218            Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([(
219                PARQUET_FIELD_ID_META_KEY.to_string(),
220                2.to_string(),
221            )])),
222            Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([(
223                PARQUET_FIELD_ID_META_KEY.to_string(),
224                3.to_string(),
225            )])),
226        ]);
227
228        let batch1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
229            Arc::new(Int32Array::from(vec![1, 2])),
230            Arc::new(StringArray::from(vec!["Alice", "Bob"])),
231            Arc::new(StringArray::from(vec!["US", "US"])),
232        ])?;
233
234        let batch2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
235            Arc::new(Int32Array::from(vec![3, 4])),
236            Arc::new(StringArray::from(vec!["Charlie", "Dave"])),
237            Arc::new(StringArray::from(vec!["US", "US"])),
238        ])?;
239
240        // Write data to the same partition (this should work)
241        writer.write(partition_key.clone(), batch1).await?;
242        writer.write(partition_key.clone(), batch2).await?;
243
244        // Close writer and get data files
245        let data_files = writer.close().await?;
246
247        // Verify at least one file was created
248        assert!(
249            !data_files.is_empty(),
250            "Expected at least one data file to be created"
251        );
252
253        // Verify that all data files have the correct partition value
254        for data_file in &data_files {
255            assert_eq!(data_file.partition, partition_value);
256        }
257
258        Ok(())
259    }
260
261    #[tokio::test]
262    async fn test_clustered_writer_sorted_partitions() -> Result<()> {
263        let temp_dir = TempDir::new()?;
264        let file_io = FileIOBuilder::new_fs_io().build()?;
265        let location_gen = DefaultLocationGenerator::with_data_location(
266            temp_dir.path().to_str().unwrap().to_string(),
267        );
268        let file_name_gen =
269            DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet);
270
271        // Create schema with partition field
272        let schema = Arc::new(
273            crate::spec::Schema::builder()
274                .with_schema_id(1)
275                .with_fields(vec![
276                    NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(),
277                    NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(),
278                    NestedField::required(3, "region", Type::Primitive(PrimitiveType::String))
279                        .into(),
280                ])
281                .build()?,
282        );
283
284        // Create partition spec
285        let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?;
286
287        // Create partition keys for different regions (in sorted order)
288        let partition_value_asia =
289            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("ASIA"))]);
290        let partition_key_asia = crate::spec::PartitionKey::new(
291            partition_spec.clone(),
292            schema.clone(),
293            partition_value_asia.clone(),
294        );
295
296        let partition_value_eu =
297            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("EU"))]);
298        let partition_key_eu = crate::spec::PartitionKey::new(
299            partition_spec.clone(),
300            schema.clone(),
301            partition_value_eu.clone(),
302        );
303
304        let partition_value_us =
305            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]);
306        let partition_key_us = crate::spec::PartitionKey::new(
307            partition_spec.clone(),
308            schema.clone(),
309            partition_value_us.clone(),
310        );
311
312        // Create writer builder
313        let parquet_writer_builder =
314            ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone());
315
316        // Create rolling file writer builder
317        let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size(
318            parquet_writer_builder,
319            file_io.clone(),
320            location_gen,
321            file_name_gen,
322        );
323
324        // Create data file writer builder
325        let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder);
326
327        // Create clustered writer
328        let mut writer = ClusteredWriter::new(data_file_writer_builder);
329
330        // Create test data with proper field ID metadata
331        let arrow_schema = Schema::new(vec![
332            Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([(
333                PARQUET_FIELD_ID_META_KEY.to_string(),
334                1.to_string(),
335            )])),
336            Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([(
337                PARQUET_FIELD_ID_META_KEY.to_string(),
338                2.to_string(),
339            )])),
340            Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([(
341                PARQUET_FIELD_ID_META_KEY.to_string(),
342                3.to_string(),
343            )])),
344        ]);
345
346        // Create batches for different partitions (in sorted order)
347        let batch_asia = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
348            Arc::new(Int32Array::from(vec![1, 2])),
349            Arc::new(StringArray::from(vec!["Alice", "Bob"])),
350            Arc::new(StringArray::from(vec!["ASIA", "ASIA"])),
351        ])?;
352
353        let batch_eu = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
354            Arc::new(Int32Array::from(vec![3, 4])),
355            Arc::new(StringArray::from(vec!["Charlie", "Dave"])),
356            Arc::new(StringArray::from(vec!["EU", "EU"])),
357        ])?;
358
359        let batch_us = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
360            Arc::new(Int32Array::from(vec![5, 6])),
361            Arc::new(StringArray::from(vec!["Eve", "Frank"])),
362            Arc::new(StringArray::from(vec!["US", "US"])),
363        ])?;
364
365        // Write data in sorted partition order (this should work)
366        writer.write(partition_key_asia.clone(), batch_asia).await?;
367        writer.write(partition_key_eu.clone(), batch_eu).await?;
368        writer.write(partition_key_us.clone(), batch_us).await?;
369
370        // Close writer and get data files
371        let data_files = writer.close().await?;
372
373        // Verify files were created for all partitions
374        assert!(
375            data_files.len() >= 3,
376            "Expected at least 3 data files (one per partition), got {}",
377            data_files.len()
378        );
379
380        // Verify that we have files for each partition
381        let mut partitions_found = std::collections::HashSet::new();
382        for data_file in &data_files {
383            partitions_found.insert(data_file.partition.clone());
384        }
385
386        assert!(
387            partitions_found.contains(&partition_value_asia),
388            "Missing ASIA partition"
389        );
390        assert!(
391            partitions_found.contains(&partition_value_eu),
392            "Missing EU partition"
393        );
394        assert!(
395            partitions_found.contains(&partition_value_us),
396            "Missing US partition"
397        );
398
399        Ok(())
400    }
401
402    #[tokio::test]
403    async fn test_clustered_writer_unsorted_partitions_error() -> Result<()> {
404        let temp_dir = TempDir::new()?;
405        let file_io = FileIOBuilder::new_fs_io().build()?;
406        let location_gen = DefaultLocationGenerator::with_data_location(
407            temp_dir.path().to_str().unwrap().to_string(),
408        );
409        let file_name_gen =
410            DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet);
411
412        // Create schema with partition field
413        let schema = Arc::new(
414            crate::spec::Schema::builder()
415                .with_schema_id(1)
416                .with_fields(vec![
417                    NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(),
418                    NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(),
419                    NestedField::required(3, "region", Type::Primitive(PrimitiveType::String))
420                        .into(),
421                ])
422                .build()?,
423        );
424
425        // Create partition spec
426        let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?;
427
428        // Create partition keys for different regions
429        let partition_value_us =
430            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]);
431        let partition_key_us = crate::spec::PartitionKey::new(
432            partition_spec.clone(),
433            schema.clone(),
434            partition_value_us.clone(),
435        );
436
437        let partition_value_eu =
438            crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("EU"))]);
439        let partition_key_eu = crate::spec::PartitionKey::new(
440            partition_spec.clone(),
441            schema.clone(),
442            partition_value_eu.clone(),
443        );
444
445        // Create writer builder
446        let parquet_writer_builder =
447            ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone());
448
449        // Create rolling file writer builder
450        let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size(
451            parquet_writer_builder,
452            file_io.clone(),
453            location_gen,
454            file_name_gen,
455        );
456
457        // Create data file writer builder
458        let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder);
459
460        // Create clustered writer
461        let mut writer = ClusteredWriter::new(data_file_writer_builder);
462
463        // Create test data with proper field ID metadata
464        let arrow_schema = Schema::new(vec![
465            Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([(
466                PARQUET_FIELD_ID_META_KEY.to_string(),
467                1.to_string(),
468            )])),
469            Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([(
470                PARQUET_FIELD_ID_META_KEY.to_string(),
471                2.to_string(),
472            )])),
473            Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([(
474                PARQUET_FIELD_ID_META_KEY.to_string(),
475                3.to_string(),
476            )])),
477        ]);
478
479        // Create batches for different partitions
480        let batch_us = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
481            Arc::new(Int32Array::from(vec![1, 2])),
482            Arc::new(StringArray::from(vec!["Alice", "Bob"])),
483            Arc::new(StringArray::from(vec!["US", "US"])),
484        ])?;
485
486        let batch_eu = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
487            Arc::new(Int32Array::from(vec![3, 4])),
488            Arc::new(StringArray::from(vec!["Charlie", "Dave"])),
489            Arc::new(StringArray::from(vec!["EU", "EU"])),
490        ])?;
491
492        let batch_us2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![
493            Arc::new(Int32Array::from(vec![5])),
494            Arc::new(StringArray::from(vec!["Eve"])),
495            Arc::new(StringArray::from(vec!["US"])),
496        ])?;
497
498        // Write data to US partition first
499        writer.write(partition_key_us.clone(), batch_us).await?;
500
501        // Write data to EU partition (this closes US partition)
502        writer.write(partition_key_eu.clone(), batch_eu).await?;
503
504        // Try to write to US partition again - this should fail because data is not sorted
505        let result = writer.write(partition_key_us.clone(), batch_us2).await;
506
507        assert!(result.is_err(), "Expected error when writing unsorted data");
508
509        let error = result.unwrap_err();
510        assert!(
511            error.to_string().contains("The input is not sorted"),
512            "Expected 'input is not sorted' error, got: {error}"
513        );
514
515        Ok(())
516    }
517}