iceberg/arrow/
delete_file_loader.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
18use std::sync::Arc;
19
20use futures::{StreamExt, TryStreamExt};
21use parquet::arrow::ParquetRecordBatchStreamBuilder;
22
23use crate::arrow::ArrowReader;
24use crate::arrow::reader::ParquetReadOptions;
25use crate::arrow::record_batch_transformer::RecordBatchTransformerBuilder;
26use crate::io::FileIO;
27use crate::scan::{ArrowRecordBatchStream, FileScanTaskDeleteFile};
28use crate::spec::{Schema, SchemaRef};
29use crate::{Error, ErrorKind, Result};
30
31/// Delete File Loader
32#[allow(unused)]
33#[async_trait::async_trait]
34pub trait DeleteFileLoader {
35    /// Read the delete file referred to in the task
36    ///
37    /// Returns the contents of the delete file as a RecordBatch stream. Applies schema evolution.
38    async fn read_delete_file(
39        &self,
40        task: &FileScanTaskDeleteFile,
41        schema: SchemaRef,
42    ) -> Result<ArrowRecordBatchStream>;
43}
44
45#[derive(Clone, Debug)]
46pub(crate) struct BasicDeleteFileLoader {
47    file_io: FileIO,
48}
49
50#[allow(unused_variables)]
51impl BasicDeleteFileLoader {
52    pub fn new(file_io: FileIO) -> Self {
53        BasicDeleteFileLoader { file_io }
54    }
55    /// Loads a RecordBatchStream for a given datafile.
56    pub(crate) async fn parquet_to_batch_stream(
57        &self,
58        data_file_path: &str,
59        file_size_in_bytes: u64,
60    ) -> Result<ArrowRecordBatchStream> {
61        /*
62           Essentially a super-cut-down ArrowReader. We can't use ArrowReader directly
63           as that introduces a circular dependency.
64        */
65        let parquet_read_options = ParquetReadOptions::builder().build();
66
67        let (parquet_file_reader, arrow_metadata) = ArrowReader::open_parquet_file(
68            data_file_path,
69            &self.file_io,
70            file_size_in_bytes,
71            parquet_read_options,
72        )
73        .await?;
74
75        let record_batch_stream =
76            ParquetRecordBatchStreamBuilder::new_with_metadata(parquet_file_reader, arrow_metadata)
77                .build()?
78                .map_err(|e| Error::new(ErrorKind::Unexpected, format!("{e}")));
79
80        Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream)
81    }
82
83    /// Evolves the schema of the RecordBatches from an equality delete file.
84    ///
85    /// Per the [Iceberg spec](https://iceberg.apache.org/spec/#equality-delete-files),
86    /// only evolves the specified `equality_ids` columns, not all table columns.
87    pub(crate) async fn evolve_schema(
88        record_batch_stream: ArrowRecordBatchStream,
89        target_schema: Arc<Schema>,
90        equality_ids: &[i32],
91    ) -> Result<ArrowRecordBatchStream> {
92        let mut record_batch_transformer =
93            RecordBatchTransformerBuilder::new(target_schema.clone(), equality_ids).build();
94
95        let record_batch_stream = record_batch_stream.map(move |record_batch| {
96            record_batch.and_then(|record_batch| {
97                record_batch_transformer.process_record_batch(record_batch)
98            })
99        });
100
101        Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream)
102    }
103}
104
105#[async_trait::async_trait]
106impl DeleteFileLoader for BasicDeleteFileLoader {
107    async fn read_delete_file(
108        &self,
109        task: &FileScanTaskDeleteFile,
110        schema: SchemaRef,
111    ) -> Result<ArrowRecordBatchStream> {
112        let raw_batch_stream = self
113            .parquet_to_batch_stream(&task.file_path, task.file_size_in_bytes)
114            .await?;
115
116        // For equality deletes, only evolve the equality_ids columns.
117        // For positional deletes (equality_ids is None), use all field IDs.
118        let field_ids = match &task.equality_ids {
119            Some(ids) => ids.clone(),
120            None => schema.field_id_to_name_map().keys().cloned().collect(),
121        };
122
123        Self::evolve_schema(raw_batch_stream, schema, &field_ids).await
124    }
125}
126
127#[cfg(test)]
128mod tests {
129    use tempfile::TempDir;
130
131    use super::*;
132    use crate::arrow::delete_filter::tests::setup;
133
134    #[tokio::test]
135    async fn test_basic_delete_file_loader_read_delete_file() {
136        let tmp_dir = TempDir::new().unwrap();
137        let table_location = tmp_dir.path();
138        let file_io = FileIO::new_with_fs();
139
140        let delete_file_loader = BasicDeleteFileLoader::new(file_io.clone());
141
142        let file_scan_tasks = setup(table_location);
143
144        let result = delete_file_loader
145            .read_delete_file(
146                &file_scan_tasks[0].deletes[0],
147                file_scan_tasks[0].schema_ref(),
148            )
149            .await
150            .unwrap();
151
152        let result = result.try_collect::<Vec<_>>().await.unwrap();
153
154        assert_eq!(result.len(), 1);
155    }
156}