Skip to content

Commit 3b393d3

Browse files
committed
make arc drop in the single thread finally.
1 parent 2b56774 commit 3b393d3

File tree

1 file changed

+30
-8
lines changed

1 file changed

+30
-8
lines changed

datafusion/core/src/datasource/physical_plan/file_stream.rs

+30-8
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
//! Note: Most traits here need to be marked `Sync + Send` to be
2222
//! compliant with the `SendableRecordBatchStream` trait.
2323
24-
use std::collections::VecDeque;
2524
use std::mem;
2625
use std::pin::Pin;
2726
use std::task::{Context, Poll};
@@ -38,12 +37,15 @@ use crate::physical_plan::RecordBatchStream;
3837
use arrow::datatypes::SchemaRef;
3938
use arrow::error::ArrowError;
4039
use arrow::record_batch::RecordBatch;
40+
use chrono::TimeZone;
4141
use datafusion_common::instant::Instant;
4242
use datafusion_common::ScalarValue;
4343

4444
use futures::future::BoxFuture;
4545
use futures::stream::BoxStream;
4646
use futures::{ready, FutureExt, Stream, StreamExt};
47+
use object_store::path::Path;
48+
use object_store::ObjectMeta;
4749

4850
/// A fallible future that resolves to a stream of [`RecordBatch`]
4951
pub type FileOpenFuture =
@@ -76,7 +78,8 @@ pub trait FileOpener: Unpin {
7678
/// A stream that iterates record batch by record batch, file over file.
7779
pub struct FileStream<F: FileOpener> {
7880
/// An iterator over input files.
79-
file_iter: VecDeque<PartitionedFile>,
81+
files: Vec<PartitionedFile>,
82+
cur_file_idx: usize,
8083
/// The stream schema (file schema including partition columns and after
8184
/// projection).
8285
projected_schema: SchemaRef,
@@ -263,7 +266,8 @@ impl<F: FileOpener> FileStream<F> {
263266
let files = config.file_groups[partition].clone();
264267

265268
Ok(Self {
266-
file_iter: files.into(),
269+
files,
270+
cur_file_idx: 0,
267271
projected_schema,
268272
remain: config.limit,
269273
file_opener,
@@ -289,18 +293,36 @@ impl<F: FileOpener> FileStream<F> {
289293
/// Since file opening is mostly IO (and may involve a
290294
/// bunch of sequential IO), it can be parallelized with decoding.
291295
fn start_next_file(&mut self) -> Option<Result<(FileOpenFuture, Vec<ScalarValue>)>> {
292-
let part_file = self.file_iter.pop_front()?;
296+
if self.cur_file_idx == self.files.len() {
297+
return None;
298+
}
299+
300+
let part_file = &mut self.files[self.cur_file_idx];
301+
self.cur_file_idx += 1;
302+
303+
let object_meta = mem::replace(
304+
&mut part_file.object_meta,
305+
ObjectMeta {
306+
location: Path::default(),
307+
last_modified: chrono::Utc.timestamp_nanos(0),
308+
size: 0,
309+
e_tag: None,
310+
version: None,
311+
},
312+
);
313+
314+
let partition_values = mem::take(&mut part_file.partition_values);
293315

294316
let file_meta = FileMeta {
295-
object_meta: part_file.object_meta,
296-
range: part_file.range,
297-
extensions: part_file.extensions,
317+
object_meta,
318+
range: part_file.range.clone(),
319+
extensions: part_file.extensions.clone(),
298320
};
299321

300322
Some(
301323
self.file_opener
302324
.open(file_meta)
303-
.map(|future| (future, part_file.partition_values)),
325+
.map(|future| (future, partition_values)),
304326
)
305327
}
306328

0 commit comments

Comments
 (0)