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

perf: Move row group decode off async thread for local streaming parquet scan #19828

Merged
merged 8 commits into from
Nov 17, 2024
Merged
Show file tree
Hide file tree
Changes from 3 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
81 changes: 25 additions & 56 deletions crates/polars-io/src/parquet/read/read_impl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ use crate::hive::materialize_hive_partitions;
use crate::mmap::{MmapBytesReader, ReaderBytes};
use crate::parquet::metadata::FileMetadataRef;
use crate::parquet::read::ROW_COUNT_OVERFLOW_ERR;
use crate::pl_async::get_runtime;
use crate::predicates::{apply_predicate, PhysicalIoExpr};
use crate::utils::get_reader_bytes;
use crate::utils::slice::split_slice_at_file;
Expand Down Expand Up @@ -1182,44 +1183,24 @@ impl BatchedParquetReader {
.fetch_row_groups(row_group_range.clone())
.await?;

let mut dfs = match store {
ColumnStore::Local(_) => rg_to_dfs(
&store,
&mut self.rows_read,
row_group_range.start,
row_group_range.end,
self.slice,
&self.metadata,
&self.schema,
self.predicate.as_deref(),
self.row_index.clone(),
self.parallel,
&self.projection,
self.use_statistics,
self.hive_partition_columns.as_deref(),
),
#[cfg(feature = "async")]
ColumnStore::Fetched(b) => {
// This branch we spawn the decoding and decompression of the bytes on a rayon task.
// This will ensure we don't block the async thread.

// Reconstruct as that makes it a 'static.
let store = ColumnStore::Fetched(b);
let (tx, rx) = tokio::sync::oneshot::channel();

// Make everything 'static.
let mut rows_read = self.rows_read;
let row_index = self.row_index.clone();
let predicate = self.predicate.clone();
let schema = self.schema.clone();
let metadata = self.metadata.clone();
let parallel = self.parallel;
let projection = self.projection.clone();
let use_statistics = self.use_statistics;
let hive_partition_columns = self.hive_partition_columns.clone();
let slice = self.slice;

let f = move || {
let mut dfs = {
// Spawn the decoding and decompression of the bytes on a rayon task.
// This will ensure we don't block the async thread.

// Make everything 'static.
let mut rows_read = self.rows_read;
let row_index = self.row_index.clone();
let predicate = self.predicate.clone();
let schema = self.schema.clone();
let metadata = self.metadata.clone();
let parallel = self.parallel;
let projection = self.projection.clone();
let use_statistics = self.use_statistics;
let hive_partition_columns = self.hive_partition_columns.clone();
let slice = self.slice;

let (dfs, rows_read) = get_runtime()
.spawn_rayon(move || {
let dfs = rg_to_dfs(
&store,
&mut rows_read,
Expand All @@ -1236,25 +1217,13 @@ impl BatchedParquetReader {
hive_partition_columns.as_deref(),
);

// Don't unwrap send attempt - async task could be cancelled.
let _ = tx.send((dfs, rows_read));
};

// Spawn the task and wait on it asynchronously.
if POOL.current_thread_index().is_some() {
// We are a rayon thread, so we can't use POOL.spawn as it would mean we spawn a task and block until
// another rayon thread executes it - we would deadlock if all rayon threads did this.
// Safety: The tokio runtime flavor is multi-threaded.
tokio::task::block_in_place(f);
} else {
POOL.spawn(f);
};
dfs.map(|x| (x, rows_read))
})
.await?;

let (dfs, rows_read) = rx.await.unwrap();
self.rows_read = rows_read;
dfs
},
}?;
self.rows_read = rows_read;
dfs
};

if let Some(ca) = self.include_file_path.as_mut() {
let mut max_len = 0;
Expand Down
28 changes: 28 additions & 0 deletions crates/polars-io/src/pl_async.rs
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,34 @@ impl RuntimeManager {
{
self.rt.spawn_blocking(f)
}

/// Run a task on the rayon threadpool. To avoid deadlocks, if the current thread is already a
/// rayon thread, the task is executed on the current thread after tokio's `block_in_place` is
/// used to spawn another thread to poll futures.
pub async fn spawn_rayon<F, O>(&self, func: F) -> O
Copy link
Collaborator Author

@nameexhaustion nameexhaustion Nov 17, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made this a function, but I think we won't be using it much as we move to the new streaming engine.

where
F: FnOnce() -> O + Send + Sync + 'static,
O: Send + Sync + 'static,
{
if POOL.current_thread_index().is_some() {
// We are a rayon thread, so we can't use POOL.spawn as it would mean we spawn a task and block until
// another rayon thread executes it - we would deadlock if all rayon threads did this.
// Safety: The tokio runtime flavor is multi-threaded.
tokio::task::block_in_place(func)
} else {
let (tx, rx) = tokio::sync::oneshot::channel();

let func = move || {
let out = func();
// Don't unwrap send attempt - async task could be cancelled.
let _ = tx.send(out);
};

POOL.spawn(func);

rx.await.unwrap()
}
}
}

static RUNTIME: Lazy<RuntimeManager> = Lazy::new(RuntimeManager::new);
Expand Down
Loading