Skip to content

Commit 2ff526f

Browse files
committed
refactor: DeleteFileManager -> DeleteFileIndex, moved to own file
1 parent 88f2c83 commit 2ff526f

File tree

3 files changed

+112
-94
lines changed

3 files changed

+112
-94
lines changed
+95
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,95 @@
1+
use std::future::Future;
2+
use std::pin::Pin;
3+
use std::sync::{Arc, RwLock};
4+
use std::task::{Context, Poll};
5+
6+
use futures::channel::mpsc::Receiver;
7+
use futures::{StreamExt, TryStreamExt};
8+
9+
use crate::runtime::spawn;
10+
use crate::scan::FileScanTaskDeleteFile;
11+
use crate::spec::DataFile;
12+
use crate::{Error, Result};
13+
14+
type DeleteFileIndexResult = Result<Option<Arc<Vec<FileScanTaskDeleteFile>>>>;
15+
16+
/// Safely shareable on-demand-populated delete file index.
17+
///
18+
/// Constructed during the file plan phase of a table scan from a channel that will
19+
/// receive the details of all delete files that can possibly apply to a scan.
20+
/// Asynchronously retrieves and lazily processes of all the delete files from FileIO
21+
/// concurrently whilst the plan proceeds with collating all of the applicable data files.
22+
/// Awaited on when constructing the resulting FileScanTask for each selected data file
23+
/// in order to populate the list of delete files to include in each `FileScanTask`.
24+
#[derive(Debug, Clone)]
25+
pub(crate) struct DeleteFileIndex {
26+
files: Arc<RwLock<Option<DeleteFileIndexResult>>>,
27+
}
28+
29+
#[derive(Debug, Clone)]
30+
pub(crate) struct DeleteFileIndexFuture {
31+
files: Arc<RwLock<Option<DeleteFileIndexResult>>>,
32+
}
33+
34+
impl Future for DeleteFileIndexFuture {
35+
type Output = DeleteFileIndexResult;
36+
37+
fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Self::Output> {
38+
let Ok(guard) = self.files.try_read() else {
39+
return Poll::Pending;
40+
};
41+
42+
if let Some(value) = guard.as_ref() {
43+
Poll::Ready(match value.as_ref() {
44+
Ok(deletes) => Ok(deletes.clone()),
45+
Err(err) => Err(Error::new(err.kind(), err.message())),
46+
})
47+
} else {
48+
Poll::Pending
49+
}
50+
}
51+
}
52+
53+
impl DeleteFileIndex {
54+
pub(crate) fn from_receiver(receiver: Receiver<Result<FileScanTaskDeleteFile>>) -> Self {
55+
let delete_file_stream = receiver.boxed();
56+
let files = Arc::new(RwLock::new(None));
57+
58+
// spawn a task to handle accumulating all the DeleteFiles that are streamed into the
59+
// index through the receiver channel. Update the `None` inside the `RwLock` to a `Some`
60+
// once the stream has been exhausted so that any consumers awaiting on the Future returned
61+
// by DeleteFileIndex::get_deletes_for_data_file can proceed
62+
spawn({
63+
let files = files.clone();
64+
async move {
65+
let _ = spawn(async move {
66+
let result = delete_file_stream.try_collect::<Vec<_>>().await;
67+
let result = result.map(|files| {
68+
if files.is_empty() {
69+
None
70+
} else {
71+
Some(Arc::new(files))
72+
}
73+
});
74+
75+
// Unwrap is ok here since this is the only place where a write lock
76+
// can be acquired, so the lock can't already have been poisoned
77+
let mut guard = files.write().unwrap();
78+
*guard = Some(result);
79+
})
80+
.await;
81+
}
82+
});
83+
84+
DeleteFileIndex { files }
85+
}
86+
87+
/// Asynchronously determines all the delete files that apply to the provided `DataFile`.
88+
pub(crate) fn get_deletes_for_data_file(&self, _data_file: &DataFile) -> DeleteFileIndexFuture {
89+
// TODO: filtering
90+
91+
DeleteFileIndexFuture {
92+
files: self.files.clone(),
93+
}
94+
}
95+
}

crates/iceberg/src/lib.rs

+1
Original file line numberDiff line numberDiff line change
@@ -82,5 +82,6 @@ pub mod transform;
8282
mod runtime;
8383

8484
pub mod arrow;
85+
pub(crate) mod delete_file_index;
8586
mod utils;
8687
pub mod writer;

crates/iceberg/src/scan.rs

+16-94
Original file line numberDiff line numberDiff line change
@@ -18,18 +18,16 @@
1818
//! Table scan api.
1919
2020
use std::collections::HashMap;
21-
use std::future::Future;
22-
use std::pin::Pin;
2321
use std::sync::{Arc, RwLock};
24-
use std::task::{Context, Poll};
2522

2623
use arrow_array::RecordBatch;
27-
use futures::channel::mpsc::{channel, Receiver, Sender};
24+
use futures::channel::mpsc::{channel, Sender};
2825
use futures::stream::BoxStream;
2926
use futures::{SinkExt, StreamExt, TryFutureExt, TryStreamExt};
3027
use serde::{Deserialize, Serialize};
3128

3229
use crate::arrow::ArrowReaderBuilder;
30+
use crate::delete_file_index::DeleteFileIndex;
3331
use crate::expr::visitors::expression_evaluator::ExpressionEvaluator;
3432
use crate::expr::visitors::inclusive_metrics_evaluator::InclusiveMetricsEvaluator;
3533
use crate::expr::visitors::inclusive_projection::InclusiveProjection;
@@ -39,7 +37,7 @@ use crate::io::object_cache::ObjectCache;
3937
use crate::io::FileIO;
4038
use crate::runtime::spawn;
4139
use crate::spec::{
42-
DataContentType, DataFile, DataFileFormat, ManifestContentType, ManifestEntryRef, ManifestFile,
40+
DataContentType, DataFileFormat, ManifestContentType, ManifestEntryRef, ManifestFile,
4341
ManifestList, Schema, SchemaRef, SnapshotRef, TableMetadataRef,
4442
};
4543
use crate::table::Table;
@@ -368,9 +366,9 @@ impl TableScan {
368366
// used to stream the results back to the caller
369367
let (file_scan_task_tx, file_scan_task_rx) = channel(concurrency_limit_manifest_entries);
370368

371-
// used to stream delete files into the DeleteFileManager
369+
// used to stream delete files into the DeleteFileIndex
372370
let (delete_file_tx, delete_file_rx) = channel(concurrency_limit_manifest_entries);
373-
let delete_file_manager = DeleteFileManager::from_receiver(delete_file_rx);
371+
let delete_file_index = DeleteFileIndex::from_receiver(delete_file_rx);
374372

375373
let manifest_list = self.plan_context.get_manifest_list().await?;
376374

@@ -381,7 +379,7 @@ impl TableScan {
381379
manifest_list,
382380
manifest_entry_data_ctx_tx,
383381
manifest_entry_delete_ctx_tx,
384-
delete_file_manager.clone(),
382+
delete_file_index.clone(),
385383
)?;
386384

387385
let mut channel_for_manifest_error = file_scan_task_tx.clone();
@@ -593,7 +591,7 @@ struct ManifestFileContext {
593591
object_cache: Arc<ObjectCache>,
594592
snapshot_schema: SchemaRef,
595593
expression_evaluator_cache: Arc<ExpressionEvaluatorCache>,
596-
delete_file_manager: DeleteFileManager,
594+
delete_file_index: DeleteFileIndex,
597595
}
598596

599597
/// Wraps a [`ManifestEntryRef`] alongside the objects that are needed
@@ -606,7 +604,7 @@ struct ManifestEntryContext {
606604
bound_predicates: Option<Arc<BoundPredicates>>,
607605
partition_spec_id: i32,
608606
snapshot_schema: SchemaRef,
609-
delete_file_manager: DeleteFileManager,
607+
delete_file_index: DeleteFileIndex,
610608
}
611609

612610
impl ManifestFileContext {
@@ -621,7 +619,7 @@ impl ManifestFileContext {
621619
field_ids,
622620
mut sender,
623621
expression_evaluator_cache,
624-
delete_file_manager,
622+
delete_file_index,
625623
..
626624
} = self;
627625

@@ -636,7 +634,7 @@ impl ManifestFileContext {
636634
partition_spec_id: manifest_file.partition_spec_id,
637635
bound_predicates: bound_predicates.clone(),
638636
snapshot_schema: snapshot_schema.clone(),
639-
delete_file_manager: delete_file_manager.clone(),
637+
delete_file_index: delete_file_index.clone(),
640638
};
641639

642640
sender
@@ -654,7 +652,7 @@ impl ManifestEntryContext {
654652
/// created from it
655653
async fn into_file_scan_task(self) -> Result<FileScanTask> {
656654
let deletes = self
657-
.delete_file_manager
655+
.delete_file_index
658656
.get_deletes_for_data_file(self.manifest_entry.data_file())
659657
.await?;
660658

@@ -712,7 +710,7 @@ impl PlanContext {
712710
manifest_list: Arc<ManifestList>,
713711
sender_data: Sender<ManifestEntryContext>,
714712
sender_delete: Sender<ManifestEntryContext>,
715-
delete_file_manager: DeleteFileManager,
713+
delete_file_index: DeleteFileIndex,
716714
) -> Result<Box<impl Iterator<Item = Result<ManifestFileContext>>>> {
717715
let manifest_files = manifest_list.entries().iter();
718716

@@ -741,7 +739,7 @@ impl PlanContext {
741739
} else {
742740
sender_delete.clone()
743741
},
744-
delete_file_manager.clone(),
742+
delete_file_index.clone(),
745743
);
746744

747745
filtered_mfcs.push(Ok(mfc));
@@ -757,7 +755,7 @@ impl PlanContext {
757755
} else {
758756
sender_delete.clone()
759757
},
760-
delete_file_manager.clone(),
758+
delete_file_index.clone(),
761759
);
762760

763761
filtered_mfcs.push(Ok(mfc));
@@ -772,7 +770,7 @@ impl PlanContext {
772770
manifest_file: &ManifestFile,
773771
partition_filter: Option<Arc<BoundPredicate>>,
774772
sender: Sender<ManifestEntryContext>,
775-
delete_file_manager: DeleteFileManager,
773+
delete_file_index: DeleteFileIndex,
776774
) -> ManifestFileContext {
777775
let bound_predicates =
778776
if let (Some(ref partition_bound_predicate), Some(snapshot_bound_predicate)) =
@@ -794,7 +792,7 @@ impl PlanContext {
794792
snapshot_schema: self.snapshot_schema.clone(),
795793
field_ids: self.field_ids.clone(),
796794
expression_evaluator_cache: self.expression_evaluator_cache.clone(),
797-
delete_file_manager,
795+
delete_file_index,
798796
}
799797
}
800798
}
@@ -1077,82 +1075,6 @@ impl FileScanTask {
10771075
}
10781076
}
10791077

1080-
type DeleteFileManagerResult = Result<Option<Arc<Vec<FileScanTaskDeleteFile>>>>;
1081-
1082-
/// Manages async retrieval of all the delete files from FileIO that are
1083-
/// applicable to the scan. Provides references to them for inclusion within FileScanTasks
1084-
#[derive(Debug, Clone)]
1085-
struct DeleteFileManager {
1086-
files: Arc<RwLock<Option<DeleteFileManagerResult>>>,
1087-
}
1088-
1089-
#[derive(Debug, Clone)]
1090-
struct DeleteFileManagerFuture {
1091-
files: Arc<RwLock<Option<DeleteFileManagerResult>>>,
1092-
}
1093-
1094-
impl Future for DeleteFileManagerFuture {
1095-
type Output = DeleteFileManagerResult;
1096-
1097-
fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Self::Output> {
1098-
let Ok(guard) = self.files.try_read() else {
1099-
return Poll::Pending;
1100-
};
1101-
1102-
if let Some(value) = guard.as_ref() {
1103-
Poll::Ready(match value.as_ref() {
1104-
Ok(deletes) => Ok(deletes.clone()),
1105-
Err(err) => Err(Error::new(err.kind(), err.message())),
1106-
})
1107-
} else {
1108-
Poll::Pending
1109-
}
1110-
}
1111-
}
1112-
1113-
impl DeleteFileManager {
1114-
pub(crate) fn from_receiver(receiver: Receiver<Result<FileScanTaskDeleteFile>>) -> Self {
1115-
let delete_file_stream = receiver.boxed();
1116-
let files = Arc::new(RwLock::new(None));
1117-
1118-
spawn({
1119-
let files = files.clone();
1120-
async move {
1121-
let _ = spawn(async move {
1122-
let result = delete_file_stream.try_collect::<Vec<_>>().await;
1123-
let result = result.map(|files| {
1124-
if files.is_empty() {
1125-
None
1126-
} else {
1127-
Some(Arc::new(files))
1128-
}
1129-
});
1130-
1131-
// Unwrap is ok here since this is the only place where a write lock
1132-
// can be acquired, so the lock can't already have been poisoned
1133-
let mut guard = files.write().unwrap();
1134-
*guard = Some(result);
1135-
})
1136-
.await;
1137-
}
1138-
});
1139-
1140-
DeleteFileManager { files }
1141-
}
1142-
1143-
pub(crate) fn get_deletes_for_data_file(
1144-
&self,
1145-
_data_file: &DataFile,
1146-
) -> DeleteFileManagerFuture {
1147-
// TODO: in the future we may want to filter out delete files
1148-
// that are not applicable to the DataFile?
1149-
1150-
DeleteFileManagerFuture {
1151-
files: self.files.clone(),
1152-
}
1153-
}
1154-
}
1155-
11561078
#[cfg(test)]
11571079
mod tests {
11581080
use std::collections::HashMap;

0 commit comments

Comments
 (0)