Skip to content

Commit 944ef3d

Browse files
authored
Add streaming JSON and CSV reading, `NewlineDelimitedStream' (#2935) (#2936)
* Add streaming JSON and CSV (#2935) * Add license header * Review feedback * Add license header * Review feedback
1 parent b772c6d commit 944ef3d

File tree

5 files changed

+463
-12
lines changed

5 files changed

+463
-12
lines changed
Lines changed: 140 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,140 @@
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+
18+
use async_trait::async_trait;
19+
use bytes::Bytes;
20+
use futures::stream::BoxStream;
21+
use futures::StreamExt;
22+
use object_store::path::Path;
23+
use object_store::Result;
24+
use object_store::{GetResult, ListResult, ObjectMeta, ObjectStore};
25+
use std::fmt::{Debug, Display, Formatter};
26+
use std::ops::Range;
27+
use std::sync::Arc;
28+
29+
/// Wraps a [`ObjectStore`] and makes its get response return chunks
30+
///
31+
/// TODO: Upstream into object_store_rs
32+
#[derive(Debug)]
33+
pub struct ChunkedStore {
34+
inner: Arc<dyn ObjectStore>,
35+
chunk_size: usize,
36+
}
37+
38+
impl ChunkedStore {
39+
pub fn new(inner: Arc<dyn ObjectStore>, chunk_size: usize) -> Self {
40+
Self { inner, chunk_size }
41+
}
42+
}
43+
44+
impl Display for ChunkedStore {
45+
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
46+
write!(f, "ChunkedStore({})", self.inner)
47+
}
48+
}
49+
50+
#[async_trait]
51+
impl ObjectStore for ChunkedStore {
52+
async fn put(&self, location: &Path, bytes: Bytes) -> Result<()> {
53+
self.inner.put(location, bytes).await
54+
}
55+
56+
async fn get(&self, location: &Path) -> Result<GetResult> {
57+
let bytes = self.inner.get(location).await?.bytes().await?;
58+
let mut offset = 0;
59+
let chunk_size = self.chunk_size;
60+
61+
Ok(GetResult::Stream(
62+
futures::stream::iter(std::iter::from_fn(move || {
63+
let remaining = bytes.len() - offset;
64+
if remaining == 0 {
65+
return None;
66+
}
67+
let to_read = remaining.min(chunk_size);
68+
let next_offset = offset + to_read;
69+
let slice = bytes.slice(offset..next_offset);
70+
offset = next_offset;
71+
Some(Ok(slice))
72+
}))
73+
.boxed(),
74+
))
75+
}
76+
77+
async fn get_range(&self, location: &Path, range: Range<usize>) -> Result<Bytes> {
78+
self.inner.get_range(location, range).await
79+
}
80+
81+
async fn head(&self, location: &Path) -> Result<ObjectMeta> {
82+
self.inner.head(location).await
83+
}
84+
85+
async fn delete(&self, location: &Path) -> Result<()> {
86+
self.inner.delete(location).await
87+
}
88+
89+
async fn list(
90+
&self,
91+
prefix: Option<&Path>,
92+
) -> Result<BoxStream<'_, Result<ObjectMeta>>> {
93+
self.inner.list(prefix).await
94+
}
95+
96+
async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result<ListResult> {
97+
self.inner.list_with_delimiter(prefix).await
98+
}
99+
100+
async fn copy(&self, from: &Path, to: &Path) -> Result<()> {
101+
self.inner.copy(from, to).await
102+
}
103+
104+
async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> {
105+
self.inner.copy_if_not_exists(from, to).await
106+
}
107+
}
108+
109+
#[cfg(test)]
110+
mod tests {
111+
use super::*;
112+
use object_store::memory::InMemory;
113+
114+
#[tokio::test]
115+
async fn test_chunked() {
116+
let location = Path::parse("test").unwrap();
117+
let store = Arc::new(InMemory::new());
118+
store
119+
.put(&location, Bytes::from(vec![0; 1001]))
120+
.await
121+
.unwrap();
122+
123+
for chunk_size in [10, 20, 31] {
124+
let store = ChunkedStore::new(store.clone(), chunk_size);
125+
let mut s = match store.get(&location).await.unwrap() {
126+
GetResult::Stream(s) => s,
127+
_ => unreachable!(),
128+
};
129+
130+
let mut remaining = 1001;
131+
while let Some(next) = s.next().await {
132+
let size = next.unwrap().len();
133+
let expected = remaining.min(chunk_size);
134+
assert_eq!(size, expected);
135+
remaining -= expected;
136+
}
137+
assert_eq!(remaining, 0);
138+
}
139+
}
140+
}

datafusion/core/src/physical_plan/file_format/csv.rs

Lines changed: 48 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ use crate::physical_plan::{
2525
};
2626

2727
use crate::datasource::listing::FileRange;
28+
use crate::physical_plan::file_format::delimited_stream::newline_delimited_stream;
2829
use crate::physical_plan::file_format::file_stream::{
2930
FileStream, FormatReader, ReaderFuture,
3031
};
@@ -167,12 +168,12 @@ struct CsvConfig {
167168
}
168169

169170
impl CsvConfig {
170-
fn open<R: std::io::Read>(&self, reader: R) -> csv::Reader<R> {
171+
fn open<R: std::io::Read>(&self, reader: R, first_chunk: bool) -> csv::Reader<R> {
171172
let datetime_format = None;
172173
csv::Reader::new(
173174
reader,
174175
Arc::clone(&self.file_schema),
175-
self.has_header,
176+
self.has_header && first_chunk,
176177
Some(self.delimiter),
177178
self.batch_size,
178179
None,
@@ -197,11 +198,18 @@ impl FormatReader for CsvOpener {
197198
Box::pin(async move {
198199
match store.get(&file.location).await? {
199200
GetResult::File(file, _) => {
200-
Ok(futures::stream::iter(config.open(file)).boxed())
201+
Ok(futures::stream::iter(config.open(file, true)).boxed())
201202
}
202-
r @ GetResult::Stream(_) => {
203-
let bytes = r.bytes().await?;
204-
Ok(futures::stream::iter(config.open(bytes.reader())).boxed())
203+
GetResult::Stream(s) => {
204+
let mut first_chunk = true;
205+
Ok(newline_delimited_stream(s.map_err(Into::into))
206+
.map_ok(move |bytes| {
207+
let reader = config.open(bytes.reader(), first_chunk);
208+
first_chunk = false;
209+
futures::stream::iter(reader)
210+
})
211+
.try_flatten()
212+
.boxed())
205213
}
206214
}
207215
})
@@ -249,12 +257,14 @@ pub async fn plan_to_csv(
249257
#[cfg(test)]
250258
mod tests {
251259
use super::*;
260+
use crate::physical_plan::file_format::chunked_store::ChunkedStore;
252261
use crate::prelude::*;
253262
use crate::test::partitioned_csv_config;
254263
use crate::test_util::aggr_test_schema_with_missing_col;
255264
use crate::{scalar::ScalarValue, test_util::aggr_test_schema};
256265
use arrow::datatypes::*;
257266
use futures::StreamExt;
267+
use object_store::local::LocalFileSystem;
258268
use std::fs::File;
259269
use std::io::Write;
260270
use tempfile::TempDir;
@@ -441,6 +451,38 @@ mod tests {
441451
Ok(schema)
442452
}
443453

454+
#[tokio::test]
455+
async fn test_chunked() {
456+
let ctx = SessionContext::new();
457+
let chunk_sizes = [10, 20, 30, 40];
458+
459+
for chunk_size in chunk_sizes {
460+
ctx.runtime_env().register_object_store(
461+
"file",
462+
"",
463+
Arc::new(ChunkedStore::new(
464+
Arc::new(LocalFileSystem::new()),
465+
chunk_size,
466+
)),
467+
);
468+
469+
let task_ctx = ctx.task_ctx();
470+
471+
let filename = "aggregate_test_100.csv";
472+
let file_schema = aggr_test_schema();
473+
let config =
474+
partitioned_csv_config(filename, file_schema.clone(), 1).unwrap();
475+
let csv = CsvExec::new(config, true, b',');
476+
477+
let it = csv.execute(0, task_ctx).unwrap();
478+
let batches: Vec<_> = it.try_collect().await.unwrap();
479+
480+
let total_rows = batches.iter().map(|b| b.num_rows()).sum::<usize>();
481+
482+
assert_eq!(total_rows, 100);
483+
}
484+
}
485+
444486
#[tokio::test]
445487
async fn write_csv_results() -> Result<()> {
446488
// create partitioned input file and context

0 commit comments

Comments
 (0)