-
Notifications
You must be signed in to change notification settings - Fork 33
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Introduces the idea of a `driver` as a pluggable way of orchestrating I/O and CPU based work. An `ExecDriver` takes a `Stream<Future<T>>` and returns a `Stream<T>`. The abstraction means we can implement drivers that use the current runtime thread (Inline), spawn the work onto an explicit Runtime, or spawn the work onto a thread pool. The `IoDriver` takes a stream of segment requests and has to resolve them. For now, we have a FileIoDriver that assumes the bytes are laid out on disk (and therefore coalescing is desirable), so it pulls requests off the queue, coalesces them, then launches concurrent I/O requests to resolve them. The VortexFile remains generic over the I/O driver, meaning the send-ness of the resulting ArrayStream is inferred based on the send-ness of the I/O driver (in the case of FileDriver, that is the send-ness of the VortexReadAt). Part of #1676
- Loading branch information
Showing
22 changed files
with
559 additions
and
213 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,19 @@ | ||
use futures_util::future::BoxFuture; | ||
use futures_util::stream::BoxStream; | ||
use futures_util::StreamExt; | ||
use vortex_array::ArrayData; | ||
use vortex_error::VortexResult; | ||
|
||
use crate::v2::exec::ExecDriver; | ||
|
||
/// An execution driver that awaits the futures inline using the caller's runtime. | ||
pub struct InlineDriver; | ||
|
||
impl ExecDriver for InlineDriver { | ||
fn drive( | ||
&self, | ||
stream: BoxStream<'static, BoxFuture<'static, VortexResult<ArrayData>>>, | ||
) -> BoxStream<'static, VortexResult<ArrayData>> { | ||
stream.then(|future| future).boxed() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,23 @@ | ||
pub mod inline; | ||
pub mod tokio; | ||
|
||
use futures_util::future::BoxFuture; | ||
use futures_util::stream::BoxStream; | ||
use vortex_array::ArrayData; | ||
use vortex_error::VortexResult; | ||
|
||
/// An execution driver is used to drive the execution of the scan operation. | ||
/// | ||
/// It is passed a stream of futures that (typically) process a single split of the file. | ||
/// Drivers are able to control the concurrency of the execution with [`futures::stream::buffered`], | ||
/// as well as _where_ the futures are executed by spawning them onto a specific runtime or thread | ||
/// pool. | ||
/// | ||
/// Note that the futures encapsulate heavy CPU code such as filtering and decompression. To | ||
/// offload keep I/O work separate, please see the [`crate::v2::io::IoDriver`] trait. | ||
pub trait ExecDriver { | ||
fn drive( | ||
&self, | ||
stream: BoxStream<'static, BoxFuture<'static, VortexResult<ArrayData>>>, | ||
) -> BoxStream<'static, VortexResult<ArrayData>>; | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
use futures_util::future::BoxFuture; | ||
use futures_util::stream::BoxStream; | ||
use futures_util::StreamExt; | ||
use tokio::runtime::Handle; | ||
use vortex_array::ArrayData; | ||
use vortex_error::{vortex_err, VortexResult}; | ||
|
||
use crate::v2::exec::ExecDriver; | ||
|
||
/// An execution driver that spawns the futures onto a Tokio runtime. | ||
pub struct TokioDriver { | ||
handle: Handle, | ||
concurrency: usize, | ||
} | ||
|
||
impl TokioDriver { | ||
pub fn new(handle: Handle, concurrency: usize) -> Self { | ||
Self { | ||
handle, | ||
concurrency, | ||
} | ||
} | ||
} | ||
|
||
impl ExecDriver for TokioDriver { | ||
fn drive( | ||
&self, | ||
stream: BoxStream<'static, BoxFuture<'static, VortexResult<ArrayData>>>, | ||
) -> BoxStream<'static, VortexResult<ArrayData>> { | ||
let handle = self.handle.clone(); | ||
stream | ||
.map(move |future| handle.spawn(future)) | ||
.buffered(self.concurrency) | ||
.map(|result| match result { | ||
Ok(result) => result, | ||
Err(e) => Err(vortex_err!("Failed to join Tokio result {}", e)), | ||
}) | ||
.boxed() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.