Skip to content

Commit 361ec59

Browse files
committed
refactor: remove concurrency when processing individual ManifestEntries in a Manifest
1 parent e135b3a commit 361ec59

File tree

2 files changed

+70
-159
lines changed

2 files changed

+70
-159
lines changed

crates/iceberg/src/arrow/reader.rs

Lines changed: 36 additions & 119 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ use crate::arrow::{arrow_schema_to_schema, get_arrow_datum};
2121
use crate::expr::visitors::bound_predicate_visitor::{visit, BoundPredicateVisitor};
2222
use crate::expr::{BoundPredicate, BoundReference};
2323
use crate::io::{FileIO, FileMetadata, FileRead};
24-
use crate::scan::FileScanTask;
2524
use crate::scan::{ArrowRecordBatchStream, FileScanTaskStream};
2625
use crate::spec::{Datum, SchemaRef};
2726
use crate::Result;
@@ -30,13 +29,12 @@ use arrow_arith::boolean::{and, is_not_null, is_null, not, or};
3029
use arrow_array::{ArrayRef, BooleanArray, RecordBatch};
3130
use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq};
3231
use arrow_schema::{ArrowError, DataType, SchemaRef as ArrowSchemaRef};
32+
use async_stream::try_stream;
3333
use bytes::Bytes;
3434
use fnv::FnvHashSet;
35-
use futures::channel::mpsc::{channel, Sender};
3635
use futures::future::BoxFuture;
3736
use futures::stream::StreamExt;
3837
use futures::{try_join, TryFutureExt};
39-
use futures::{SinkExt, TryStreamExt};
4038
use parquet::arrow::arrow_reader::{ArrowPredicateFn, RowFilter};
4139
use parquet::arrow::async_reader::{AsyncFileReader, MetadataLoader};
4240
use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask, PARQUET_FIELD_ID_META_KEY};
@@ -46,15 +44,11 @@ use std::collections::{HashMap, HashSet};
4644
use std::ops::Range;
4745
use std::str::FromStr;
4846
use std::sync::Arc;
49-
use tokio::spawn;
50-
51-
const CHANNEL_BUFFER_SIZE: usize = 10;
52-
const CONCURRENCY_LIMIT_TASKS: usize = 10;
5347

5448
/// Builder to create ArrowReader
5549
pub struct ArrowReaderBuilder {
5650
batch_size: Option<usize>,
57-
field_ids: Arc<Vec<usize>>,
51+
field_ids: Vec<usize>,
5852
file_io: FileIO,
5953
schema: SchemaRef,
6054
predicate: Option<BoundPredicate>,
@@ -65,7 +59,7 @@ impl ArrowReaderBuilder {
6559
pub fn new(file_io: FileIO, schema: SchemaRef) -> Self {
6660
ArrowReaderBuilder {
6761
batch_size: None,
68-
field_ids: Arc::new(vec![]),
62+
field_ids: vec![],
6963
file_io,
7064
schema,
7165
predicate: None,
@@ -81,10 +75,7 @@ impl ArrowReaderBuilder {
8175

8276
/// Sets the desired column projection with a list of field ids.
8377
pub fn with_field_ids(mut self, field_ids: impl IntoIterator<Item = usize>) -> Self {
84-
let field_ids = field_ids.into_iter().collect();
85-
let field_ids_arc = Arc::new(field_ids);
86-
self.field_ids = field_ids_arc;
87-
78+
self.field_ids = field_ids.into_iter().collect();
8879
self
8980
}
9081

@@ -109,7 +100,7 @@ impl ArrowReaderBuilder {
109100
/// Reads data from Parquet files
110101
pub struct ArrowReader {
111102
batch_size: Option<usize>,
112-
field_ids: Arc<Vec<usize>>,
103+
field_ids: Vec<usize>,
113104
#[allow(dead_code)]
114105
schema: SchemaRef,
115106
file_io: FileIO,
@@ -119,9 +110,7 @@ pub struct ArrowReader {
119110
impl ArrowReader {
120111
/// Take a stream of FileScanTasks and reads all the files.
121112
/// Returns a stream of Arrow RecordBatches containing the data from the files
122-
pub fn read(self, tasks: FileScanTaskStream) -> Result<ArrowRecordBatchStream> {
123-
let (sender, receiver) = channel(CHANNEL_BUFFER_SIZE);
124-
113+
pub fn read(self, mut tasks: FileScanTaskStream) -> Result<ArrowRecordBatchStream> {
125114
// Collect Parquet column indices from field ids
126115
let mut collector = CollectFieldIdVisitor {
127116
field_ids: HashSet::default(),
@@ -130,116 +119,44 @@ impl ArrowReader {
130119
visit(&mut collector, predicates)?;
131120
}
132121

133-
let tasks = tasks.map(move |task| self.build_file_scan_task_context(task, sender.clone()));
134-
135-
spawn(async move {
136-
tasks
137-
.try_for_each_concurrent(CONCURRENCY_LIMIT_TASKS, Self::process_file_scan_task)
138-
.await
139-
});
140-
141-
Ok(receiver.boxed())
142-
}
143-
144-
fn build_file_scan_task_context(
145-
&self,
146-
task: Result<FileScanTask>,
147-
sender: Sender<Result<RecordBatch>>,
148-
) -> Result<FileScanTaskContext> {
149-
Ok(FileScanTaskContext::new(
150-
task?,
151-
self.file_io.clone(),
152-
sender,
153-
self.batch_size,
154-
self.field_ids.clone(),
155-
self.schema.clone(),
156-
self.predicate.clone(),
157-
))
158-
}
159-
160-
async fn process_file_scan_task(mut context: FileScanTaskContext) -> Result<()> {
161-
let file_scan_task = context.take_task();
162-
163-
// Collect Parquet column indices from field ids
164-
let mut collector = CollectFieldIdVisitor {
165-
field_ids: HashSet::default(),
166-
};
167-
if let Some(predicate) = &context.predicate {
168-
visit(&mut collector, predicate)?;
169-
}
170-
171-
let parquet_file = context
172-
.file_io
173-
.new_input(file_scan_task.data().data_file().file_path())?;
174-
let (parquet_metadata, parquet_reader) =
175-
try_join!(parquet_file.metadata(), parquet_file.reader())?;
176-
let arrow_file_reader = ArrowFileReader::new(parquet_metadata, parquet_reader);
177-
178-
let mut batch_stream_builder =
179-
ParquetRecordBatchStreamBuilder::new(arrow_file_reader).await?;
122+
Ok(try_stream! {
123+
while let Some(Ok(task)) = tasks.next().await {
180124

181-
let parquet_schema = batch_stream_builder.parquet_schema();
182-
let arrow_schema = batch_stream_builder.schema();
125+
let parquet_file = self
126+
.file_io
127+
.new_input(task.data().data_file().file_path())?;
128+
let (parquet_metadata, parquet_reader) =
129+
try_join!(parquet_file.metadata(), parquet_file.reader())?;
130+
let arrow_file_reader = ArrowFileReader::new(parquet_metadata, parquet_reader);
183131

184-
let projection_mask = context.get_arrow_projection_mask(parquet_schema, arrow_schema)?;
185-
batch_stream_builder = batch_stream_builder.with_projection(projection_mask);
132+
let mut batch_stream_builder =
133+
ParquetRecordBatchStreamBuilder::new(arrow_file_reader).await?;
186134

187-
let parquet_schema = batch_stream_builder.parquet_schema();
188-
let row_filter = context.get_row_filter(parquet_schema, &collector)?;
189-
190-
if let Some(row_filter) = row_filter {
191-
batch_stream_builder = batch_stream_builder.with_row_filter(row_filter);
192-
}
135+
let parquet_schema = batch_stream_builder.parquet_schema();
136+
let arrow_schema = batch_stream_builder.schema();
193137

194-
if let Some(batch_size) = context.batch_size {
195-
batch_stream_builder = batch_stream_builder.with_batch_size(batch_size);
196-
}
138+
let projection_mask = self.get_arrow_projection_mask(parquet_schema, arrow_schema)?;
139+
batch_stream_builder = batch_stream_builder.with_projection(projection_mask);
197140

198-
let mut batch_stream = batch_stream_builder.build()?;
141+
let parquet_schema = batch_stream_builder.parquet_schema();
142+
let row_filter = self.get_row_filter(parquet_schema, &collector)?;
199143

200-
while let Some(batch) = batch_stream.next().await {
201-
context.sender.send(Ok(batch?)).await?;
202-
}
144+
if let Some(row_filter) = row_filter {
145+
batch_stream_builder = batch_stream_builder.with_row_filter(row_filter);
146+
}
203147

204-
Ok(())
205-
}
206-
}
148+
if let Some(batch_size) = self.batch_size {
149+
batch_stream_builder = batch_stream_builder.with_batch_size(batch_size);
150+
}
207151

208-
struct FileScanTaskContext {
209-
file_scan_task: Option<FileScanTask>,
210-
file_io: FileIO,
211-
sender: Sender<Result<RecordBatch>>,
212-
batch_size: Option<usize>,
213-
field_ids: Arc<Vec<usize>>,
214-
schema: SchemaRef,
215-
predicate: Option<BoundPredicate>,
216-
}
152+
let mut batch_stream = batch_stream_builder.build()?;
217153

218-
impl FileScanTaskContext {
219-
fn new(
220-
file_scan_task: FileScanTask,
221-
file_io: FileIO,
222-
sender: Sender<Result<RecordBatch>>,
223-
batch_size: Option<usize>,
224-
field_ids: Arc<Vec<usize>>,
225-
schema: SchemaRef,
226-
predicate: Option<BoundPredicate>,
227-
) -> Self {
228-
FileScanTaskContext {
229-
file_scan_task: Some(file_scan_task),
230-
file_io,
231-
sender,
232-
batch_size,
233-
field_ids,
234-
schema,
235-
predicate,
154+
while let Some(batch) = batch_stream.next().await {
155+
yield batch?;
156+
}
157+
}
236158
}
237-
}
238-
239-
fn take_task(&mut self) -> FileScanTask {
240-
let mut result = None;
241-
std::mem::swap(&mut self.file_scan_task, &mut result);
242-
result.unwrap()
159+
.boxed())
243160
}
244161

245162
fn get_arrow_projection_mask(
@@ -297,8 +214,8 @@ impl FileScanTaskContext {
297214
}
298215

299216
let mut indices = vec![];
300-
for field_id in self.field_ids.as_ref() {
301-
if let Some(col_idx) = column_map.get(&(*field_id as i32)) {
217+
for &field_id in &self.field_ids {
218+
if let Some(col_idx) = column_map.get(&(field_id as i32)) {
302219
indices.push(*col_idx);
303220
} else {
304221
return Err(Error::new(

crates/iceberg/src/scan.rs

Lines changed: 34 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,6 @@ use tokio::spawn;
4040

4141
const CHANNEL_BUFFER_SIZE: usize = 10;
4242
const CONCURRENCY_LIMIT_MANIFEST_FILES: usize = 10;
43-
const CONCURRENCY_LIMIT_MANIFEST_ENTRIES: usize = 10;
4443
const SUPPORTED_MANIFEST_FILE_CONTENT_TYPES: [ManifestContentType; 1] = [ManifestContentType::Data];
4544

4645
/// A stream of [`FileScanTask`].
@@ -212,13 +211,13 @@ impl TableScan {
212211
self.case_sensitive,
213212
)?;
214213

215-
let (sender, receiver) = channel(CHANNEL_BUFFER_SIZE);
216-
217214
let manifest_list = context
218215
.snapshot
219216
.load_manifest_list(&context.file_io, &context.table_metadata)
220217
.await?;
221218

219+
let (sender, receiver) = channel(CHANNEL_BUFFER_SIZE);
220+
222221
spawn(async move {
223222
let _ = ConcurrentFileScanStreamContext::new(context, sender)
224223
.run(manifest_list)
@@ -391,56 +390,51 @@ impl ConcurrentFileScanStreamContext {
391390
Arc<FileScanStreamContext>,
392391
),
393392
) -> Result<()> {
394-
let (manifest_file, file_io, sender, context) = manifest_and_file_io_and_sender;
393+
let (manifest_file, file_io, mut sender, context) = manifest_and_file_io_and_sender;
395394

396395
let manifest = manifest_file.load_manifest(&file_io).await?;
396+
for manifest_entry in manifest.entries() {
397+
if !manifest_entry.is_alive() {
398+
continue;
399+
}
397400

398-
let manifest_entries = manifest
399-
.entries()
400-
.iter()
401-
.filter(|x| x.is_alive())
402-
.map(|manifest_entry| Ok((manifest_entry, sender.clone(), context.clone())));
401+
Self::reject_unsupported_manifest_entry_content_types(manifest_entry)?;
403402

404-
futures::stream::iter(manifest_entries)
405-
.try_for_each_concurrent(
406-
CONCURRENCY_LIMIT_MANIFEST_ENTRIES,
407-
Self::process_manifest_entry,
408-
)
409-
.await
403+
if let Some(bound_predicate) = context.bound_filter() {
404+
// reject any manifest entries whose data file's metrics don't match the filter.
405+
if !InclusiveMetricsEvaluator::eval(
406+
bound_predicate,
407+
manifest_entry.data_file(),
408+
false,
409+
)? {
410+
return Ok(());
411+
}
412+
}
413+
414+
// TODO: Apply ExpressionEvaluator
415+
416+
sender
417+
.send(Ok(Self::manifest_entry_to_file_scan_task(manifest_entry)))
418+
.await?;
419+
}
420+
421+
Ok(())
410422
}
411423

412-
async fn process_manifest_entry(
413-
manifest_entry_and_sender: (
414-
&ManifestEntryRef,
415-
Sender<Result<FileScanTask>>,
416-
Arc<FileScanStreamContext>,
417-
),
424+
fn reject_unsupported_manifest_entry_content_types(
425+
manifest_entry: &ManifestEntryRef,
418426
) -> Result<()> {
419-
let (manifest_entry, mut sender, context) = manifest_entry_and_sender;
420-
421-
if !matches!(manifest_entry.content_type(), DataContentType::Data) {
422-
return Err(Error::new(
427+
if matches!(manifest_entry.content_type(), DataContentType::Data) {
428+
Ok(())
429+
} else {
430+
Err(Error::new(
423431
ErrorKind::FeatureUnsupported,
424432
format!(
425433
"Files of type '{:?}' are not supported yet.",
426434
manifest_entry.content_type()
427435
),
428-
));
429-
}
430-
431-
if let Some(bound_predicate) = context.bound_filter() {
432-
// reject any manifest entries whose data file's metrics don't match the filter.
433-
if !InclusiveMetricsEvaluator::eval(bound_predicate, manifest_entry.data_file(), false)?
434-
{
435-
return Ok(());
436-
}
436+
))
437437
}
438-
439-
// TODO: Apply ExpressionEvaluator
440-
441-
Ok(sender
442-
.send(Ok(Self::manifest_entry_to_file_scan_task(manifest_entry)))
443-
.await?)
444438
}
445439

446440
fn manifest_entry_to_file_scan_task(manifest_entry: &ManifestEntryRef) -> FileScanTask {

0 commit comments

Comments
 (0)