Skip to content

Commit 042a33b

Browse files
committed
[refacto] keep async reader stub
1 parent 2ef3233 commit 042a33b

File tree

9 files changed

+27
-13
lines changed

9 files changed

+27
-13
lines changed

datafusion/src/datasource/file_format/avro.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ impl FileFormat for AvroFormat {
6767
.object_store_registry
6868
.get_by_uri(&fmeta.path)?
6969
.file_reader(fmeta)?
70-
.reader()?;
70+
.sync_reader()?;
7171
let schema = read_avro_schema_from_reader(&mut reader)?;
7272
schemas.push(schema);
7373
}

datafusion/src/datasource/file_format/csv.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,7 @@ impl FileFormat for CsvFormat {
9595
.object_store_registry
9696
.get_by_uri(&fmeta.path)?
9797
.file_reader(fmeta)?
98-
.reader()?;
98+
.sync_reader()?;
9999
let (schema, records_read) = arrow::csv::reader::infer_reader_schema(
100100
&mut reader,
101101
self.delimiter,

datafusion/src/datasource/file_format/json.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ impl FileFormat for JsonFormat {
8181
.object_store_registry
8282
.get_by_uri(&fmeta.path)?
8383
.file_reader(fmeta)?
84-
.reader()?;
84+
.sync_reader()?;
8585
let mut reader = BufReader::new(reader);
8686
let iter = ValueIter::new(&mut reader, None);
8787
let schema = infer_json_schema_from_iterator(iter.take_while(|_| {

datafusion/src/datasource/file_format/parquet.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -337,7 +337,7 @@ impl ChunkReader for ChunkObjectReader {
337337

338338
fn get_read(&self, start: u64, length: usize) -> ParquetResult<Self::T> {
339339
self.0
340-
.chunk_reader(start, length)
340+
.sync_chunk_reader(start, length)
341341
.map_err(|e| ParquetError::ArrowError(e.to_string()))
342342
}
343343
}

datafusion/src/datasource/object_store/local.rs

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ use std::io::{Read, Seek, SeekFrom};
2222
use std::sync::Arc;
2323

2424
use async_trait::async_trait;
25-
use futures::{stream, StreamExt};
25+
use futures::{stream, AsyncRead, StreamExt};
2626

2727
use crate::datasource::object_store::{
2828
ListEntryStream, ObjectReader, ObjectStore, SizedFile, SizedFileStream,
@@ -65,7 +65,17 @@ impl LocalFileReader {
6565

6666
#[async_trait]
6767
impl ObjectReader for LocalFileReader {
68-
fn chunk_reader(
68+
async fn chunk_reader(
69+
&self,
70+
_start: u64,
71+
_length: usize,
72+
) -> Result<Box<dyn AsyncRead>> {
73+
todo!(
74+
"implement once async file readers are available (arrow-rs#78, arrow-rs#111)"
75+
)
76+
}
77+
78+
fn sync_chunk_reader(
6979
&self,
7080
start: u64,
7181
length: usize,

datafusion/src/datasource/object_store/mod.rs

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ use std::pin::Pin;
2626
use std::sync::{Arc, RwLock};
2727

2828
use async_trait::async_trait;
29-
use futures::{Stream, StreamExt};
29+
use futures::{AsyncRead, Stream, StreamExt};
3030

3131
use local::LocalFileSystem;
3232

@@ -37,16 +37,20 @@ use crate::error::{DataFusionError, Result};
3737
/// have some performance impacts.
3838
#[async_trait]
3939
pub trait ObjectReader {
40+
/// Get reader for a part [start, start + length] in the file asynchronously
41+
async fn chunk_reader(&self, start: u64, length: usize)
42+
-> Result<Box<dyn AsyncRead>>;
43+
4044
/// Get reader for a part [start, start + length] in the file
41-
fn chunk_reader(
45+
fn sync_chunk_reader(
4246
&self,
4347
start: u64,
4448
length: usize,
4549
) -> Result<Box<dyn Read + Send + Sync>>;
4650

4751
/// Get reader for the entire file
48-
fn reader(&self) -> Result<Box<dyn Read + Send + Sync>> {
49-
self.chunk_reader(0, self.length() as usize)
52+
fn sync_reader(&self) -> Result<Box<dyn Read + Send + Sync>> {
53+
self.sync_chunk_reader(0, self.length() as usize)
5054
}
5155

5256
/// Get the size of the file

datafusion/src/physical_plan/file_format/avro.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ impl ExecutionPlan for AvroExec {
129129
.object_store_registry
130130
.get_by_uri(&self.files[partition].file.path)?
131131
.file_reader(self.files[partition].file.clone())?
132-
.reader()?;
132+
.sync_reader()?;
133133

134134
let proj = self.projection.as_ref().map(|p| {
135135
p.iter()

datafusion/src/physical_plan/file_format/csv.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -196,7 +196,7 @@ impl CsvStream<Box<dyn Read + Send + Sync>> {
196196
let file = object_store_registry
197197
.get_by_uri(&file.path)?
198198
.file_reader(file.clone())?
199-
.reader()?;
199+
.sync_reader()?;
200200
Self::try_new_from_reader(
201201
file, schema, has_header, delimiter, projection, batch_size, limit,
202202
)

datafusion/src/physical_plan/file_format/json.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ impl ExecutionPlan for NdJsonExec {
129129
.object_store_registry
130130
.get_by_uri(&self.files[partition].file.path)?
131131
.file_reader(self.files[partition].file.clone())?
132-
.reader()?;
132+
.sync_reader()?;
133133

134134
let json_reader = json::Reader::new(file, self.schema(), self.batch_size, proj);
135135

0 commit comments

Comments
 (0)