Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BUG] [Parquet] Fix double-await on JoinHandles concurrency bug in Parquet reader. #1841

Merged
merged 4 commits into from
Feb 7, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions src/common/io-config/src/s3.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ impl Default for S3Config {
access_key: None,
max_connections_per_io_thread: 8,
retry_initial_backoff_ms: 1000,
connect_timeout_ms: 10_000,
read_timeout_ms: 10_000,
connect_timeout_ms: 30_000,
read_timeout_ms: 30_000,
clarkzinzow marked this conversation as resolved.
Show resolved Hide resolved
// AWS EMR actually does 100 tries by default for AIMD retries
// (See [Advanced AIMD retry settings]: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark-emrfs-retry.html)
num_tries: 25,
Expand Down
3 changes: 3 additions & 0 deletions src/daft-io/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,9 @@ pub enum Error {

#[snafu(display("Error joining spawned task: {}", source), context(false))]
JoinError { source: tokio::task::JoinError },

#[snafu(display("Cached error: {}", source))]
CachedError { source: Arc<Error> },
}

impl From<Error> for DaftError {
Expand Down
1 change: 1 addition & 0 deletions src/daft-parquet/src/lib.rs
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
#![feature(async_closure)]
#![feature(let_chains)]
#![feature(result_flattening)]
use common_error::DaftError;
use snafu::Snafu;

Expand Down
27 changes: 19 additions & 8 deletions src/daft-parquet/src/read_planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ use std::{fmt::Display, ops::Range, sync::Arc};
use bytes::Bytes;
use common_error::DaftResult;
use daft_io::{IOClient, IOStatsRef};
use futures::StreamExt;
use futures::{StreamExt, TryStreamExt};
use tokio::task::JoinHandle;

type RangeList = Vec<Range<usize>>;
Expand Down Expand Up @@ -86,7 +86,8 @@ impl ReadPlanPass for SplitLargeRequestPass {

enum RangeCacheState {
InFlight(JoinHandle<std::result::Result<Bytes, daft_io::Error>>),
Ready(Bytes),
// Ready-state stores either the fetched bytes, or a shared error if the fetch failed.
Ready(std::result::Result<Bytes, Arc<daft_io::Error>>),
}

struct RangeCacheEntry {
Expand All @@ -99,16 +100,25 @@ impl RangeCacheEntry {
async fn get_or_wait(&self, range: Range<usize>) -> std::result::Result<Bytes, daft_io::Error> {
{
let mut _guard = self.state.lock().await;
match &mut (*_guard) {
match &mut *_guard {
RangeCacheState::InFlight(f) => {
// TODO(sammy): thread in url for join error
let v = f
.await
.map_err(|e| daft_io::Error::JoinError { source: e })??;
*_guard = RangeCacheState::Ready(v.clone());
Ok(v.slice(range))
.map_err(|e| daft_io::Error::JoinError { source: e })
.flatten()
.map_err(Arc::new);
let sliced = v
.as_ref()
.map(|b| b.slice(range))
.map_err(|e| daft_io::Error::CachedError { source: e.clone() });
*_guard = RangeCacheState::Ready(v);
sliced
}
RangeCacheState::Ready(v) => Ok(v.slice(range)),
RangeCacheState::Ready(v) => v
.as_ref()
.map(|b| b.slice(range))
.map_err(|e| daft_io::Error::CachedError { source: e.clone() }),
}
}
}
Expand Down Expand Up @@ -247,7 +257,8 @@ impl RangesContainer {
assert_eq!(current_pos, range.end);

let bytes_iter = tokio_stream::iter(needed_entries.into_iter().zip(ranges_to_slice))
.then(|(e, r)| async move { e.get_or_wait(r).await });
.then(|(e, r)| async move { e.get_or_wait(r).await })
.inspect_err(|e| panic!("Reading a range of Parquet bytes failed: {}", e));

let stream_reader = tokio_util::io::StreamReader::new(bytes_iter);
let convert = async_compat::Compat::new(stream_reader);
Expand Down
42 changes: 42 additions & 0 deletions tests/integration/io/parquet/test_reads_public_data.py
Original file line number Diff line number Diff line change
Expand Up @@ -377,3 +377,45 @@ def test_row_groups_selection_into_pyarrow_bulk(public_storage_io_config, multit
for i, t in enumerate(rest):
assert len(t) == 10
assert first[i * 10 : (i + 1) * 10] == t


@pytest.mark.integration()
@pytest.mark.parametrize(
"multithreaded_io",
[False, True],
)
def test_connect_timeout(multithreaded_io):
url = "s3://daft-public-data/test_fixtures/parquet-dev/mvp.parquet"
connect_timeout_config = daft.io.IOConfig(
s3=daft.io.S3Config(
# NOTE: no keys or endpoints specified for an AWS public s3 bucket
region_name="us-west-2",
anonymous=True,
connect_timeout_ms=1,
num_tries=3,
)
)

with pytest.raises(ValueError, match="HTTP connect timeout"):
MicroPartition.read_parquet(url, io_config=connect_timeout_config, multithreaded_io=multithreaded_io).to_arrow()


@pytest.mark.integration()
@pytest.mark.parametrize(
"multithreaded_io",
[False, True],
)
def test_read_timeout(multithreaded_io):
url = "s3://daft-public-data/test_fixtures/parquet-dev/mvp.parquet"
read_timeout_config = daft.io.IOConfig(
s3=daft.io.S3Config(
# NOTE: no keys or endpoints specified for an AWS public s3 bucket
region_name="us-west-2",
anonymous=True,
read_timeout_ms=1,
num_tries=3,
)
)

with pytest.raises(ValueError, match="HTTP read timeout"):
MicroPartition.read_parquet(url, io_config=read_timeout_config, multithreaded_io=multithreaded_io).to_arrow()
Loading