datafusion_physical_plan/stream.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Stream wrappers for physical operators
use std::pin::Pin;
use std::sync::Arc;
use std::task::Context;
use std::task::Poll;
use super::metrics::BaselineMetrics;
use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream};
use crate::displayable;
use arrow::{datatypes::SchemaRef, record_batch::RecordBatch};
use datafusion_common::{internal_err, Result};
use datafusion_execution::TaskContext;
use futures::stream::BoxStream;
use futures::{Future, Stream, StreamExt};
use log::debug;
use pin_project_lite::pin_project;
use tokio::sync::mpsc::{Receiver, Sender};
use tokio::task::JoinSet;
/// Creates a stream from a collection of producing tasks, routing panics to the stream.
///
/// Note that this is similar to [`ReceiverStream` from tokio-stream], with the differences being:
///
/// 1. Methods to bound and "detach" tasks (`spawn()` and `spawn_blocking()`).
///
/// 2. Propagates panics, whereas the `tokio` version doesn't propagate panics to the receiver.
///
/// 3. Automatically cancels any outstanding tasks when the receiver stream is dropped.
///
/// [`ReceiverStream` from tokio-stream]: https://docs.rs/tokio-stream/latest/tokio_stream/wrappers/struct.ReceiverStream.html
pub(crate) struct ReceiverStreamBuilder<O> {
tx: Sender<Result<O>>,
rx: Receiver<Result<O>>,
join_set: JoinSet<Result<()>>,
}
impl<O: Send + 'static> ReceiverStreamBuilder<O> {
/// Create new channels with the specified buffer size
pub fn new(capacity: usize) -> Self {
let (tx, rx) = tokio::sync::mpsc::channel(capacity);
Self {
tx,
rx,
join_set: JoinSet::new(),
}
}
/// Get a handle for sending data to the output
pub fn tx(&self) -> Sender<Result<O>> {
self.tx.clone()
}
/// Spawn task that will be aborted if this builder (or the stream
/// built from it) are dropped
pub fn spawn<F>(&mut self, task: F)
where
F: Future<Output = Result<()>>,
F: Send + 'static,
{
self.join_set.spawn(task);
}
/// Spawn a blocking task that will be aborted if this builder (or the stream
/// built from it) are dropped.
///
/// This is often used to spawn tasks that write to the sender
/// retrieved from `Self::tx`.
pub fn spawn_blocking<F>(&mut self, f: F)
where
F: FnOnce() -> Result<()>,
F: Send + 'static,
{
self.join_set.spawn_blocking(f);
}
/// Create a stream of all data written to `tx`
pub fn build(self) -> BoxStream<'static, Result<O>> {
let Self {
tx,
rx,
mut join_set,
} = self;
// Doesn't need tx
drop(tx);
// future that checks the result of the join set, and propagates panic if seen
let check = async move {
while let Some(result) = join_set.join_next().await {
match result {
Ok(task_result) => {
match task_result {
// Nothing to report
Ok(_) => continue,
// This means a blocking task error
Err(error) => return Some(Err(error)),
}
}
// This means a tokio task error, likely a panic
Err(e) => {
if e.is_panic() {
// resume on the main thread
std::panic::resume_unwind(e.into_panic());
} else {
// This should only occur if the task is
// cancelled, which would only occur if
// the JoinSet were aborted, which in turn
// would imply that the receiver has been
// dropped and this code is not running
return Some(internal_err!("Non Panic Task error: {e}"));
}
}
}
}
None
};
let check_stream = futures::stream::once(check)
// unwrap Option / only return the error
.filter_map(|item| async move { item });
// Convert the receiver into a stream
let rx_stream = futures::stream::unfold(rx, |mut rx| async move {
let next_item = rx.recv().await;
next_item.map(|next_item| (next_item, rx))
});
// Merge the streams together so whichever is ready first
// produces the batch
futures::stream::select(rx_stream, check_stream).boxed()
}
}
/// Builder for `RecordBatchReceiverStream` that propagates errors
/// and panic's correctly.
///
/// [`RecordBatchReceiverStreamBuilder`] is used to spawn one or more tasks
/// that produce [`RecordBatch`]es and send them to a single
/// `Receiver` which can improve parallelism.
///
/// This also handles propagating panic`s and canceling the tasks.
///
/// # Example
///
/// The following example spawns 2 tasks that will write [`RecordBatch`]es to
/// the `tx` end of the builder, after building the stream, we can receive
/// those batches with calling `.next()`
///
/// ```
/// # use std::sync::Arc;
/// # use datafusion_common::arrow::datatypes::{Schema, Field, DataType};
/// # use datafusion_common::arrow::array::RecordBatch;
/// # use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder;
/// # use futures::stream::StreamExt;
/// # use tokio::runtime::Builder;
/// # let rt = Builder::new_current_thread().build().unwrap();
/// #
/// # rt.block_on(async {
/// let schema = Arc::new(Schema::new(vec![Field::new("foo", DataType::Int8, false)]));
/// let mut builder = RecordBatchReceiverStreamBuilder::new(Arc::clone(&schema), 10);
///
/// // task 1
/// let tx_1 = builder.tx();
/// let schema_1 = Arc::clone(&schema);
/// builder.spawn(async move {
/// // Your task needs to send batches to the tx
/// tx_1.send(Ok(RecordBatch::new_empty(schema_1))).await.unwrap();
///
/// Ok(())
/// });
///
/// // task 2
/// let tx_2 = builder.tx();
/// let schema_2 = Arc::clone(&schema);
/// builder.spawn(async move {
/// // Your task needs to send batches to the tx
/// tx_2.send(Ok(RecordBatch::new_empty(schema_2))).await.unwrap();
///
/// Ok(())
/// });
///
/// let mut stream = builder.build();
/// while let Some(res_batch) = stream.next().await {
/// // `res_batch` can either from task 1 or 2
///
/// // do something with `res_batch`
/// }
/// # });
/// ```
pub struct RecordBatchReceiverStreamBuilder {
schema: SchemaRef,
inner: ReceiverStreamBuilder<RecordBatch>,
}
impl RecordBatchReceiverStreamBuilder {
/// Create new channels with the specified buffer size
pub fn new(schema: SchemaRef, capacity: usize) -> Self {
Self {
schema,
inner: ReceiverStreamBuilder::new(capacity),
}
}
/// Get a handle for sending [`RecordBatch`] to the output
pub fn tx(&self) -> Sender<Result<RecordBatch>> {
self.inner.tx()
}
/// Spawn task that will be aborted if this builder (or the stream
/// built from it) are dropped
///
/// This is often used to spawn tasks that write to the sender
/// retrieved from [`Self::tx`], for examples, see the document
/// of this type.
pub fn spawn<F>(&mut self, task: F)
where
F: Future<Output = Result<()>>,
F: Send + 'static,
{
self.inner.spawn(task)
}
/// Spawn a blocking task that will be aborted if this builder (or the stream
/// built from it) are dropped
///
/// This is often used to spawn tasks that write to the sender
/// retrieved from [`Self::tx`], for examples, see the document
/// of this type.
pub fn spawn_blocking<F>(&mut self, f: F)
where
F: FnOnce() -> Result<()>,
F: Send + 'static,
{
self.inner.spawn_blocking(f)
}
/// Runs the `partition` of the `input` ExecutionPlan on the
/// tokio threadpool and writes its outputs to this stream
///
/// If the input partition produces an error, the error will be
/// sent to the output stream and no further results are sent.
pub(crate) fn run_input(
&mut self,
input: Arc<dyn ExecutionPlan>,
partition: usize,
context: Arc<TaskContext>,
) {
let output = self.tx();
self.inner.spawn(async move {
let mut stream = match input.execute(partition, context) {
Err(e) => {
// If send fails, the plan being torn down, there
// is no place to send the error and no reason to continue.
output.send(Err(e)).await.ok();
debug!(
"Stopping execution: error executing input: {}",
displayable(input.as_ref()).one_line()
);
return Ok(());
}
Ok(stream) => stream,
};
// Transfer batches from inner stream to the output tx
// immediately.
while let Some(item) = stream.next().await {
let is_err = item.is_err();
// If send fails, plan being torn down, there is no
// place to send the error and no reason to continue.
if output.send(item).await.is_err() {
debug!(
"Stopping execution: output is gone, plan cancelling: {}",
displayable(input.as_ref()).one_line()
);
return Ok(());
}
// Stop after the first error is encountered (Don't
// drive all streams to completion)
if is_err {
debug!(
"Stopping execution: plan returned error: {}",
displayable(input.as_ref()).one_line()
);
return Ok(());
}
}
Ok(())
});
}
/// Create a stream of all [`RecordBatch`] written to `tx`
pub fn build(self) -> SendableRecordBatchStream {
Box::pin(RecordBatchStreamAdapter::new(
self.schema,
self.inner.build(),
))
}
}
#[doc(hidden)]
pub struct RecordBatchReceiverStream {}
impl RecordBatchReceiverStream {
/// Create a builder with an internal buffer of capacity batches.
pub fn builder(
schema: SchemaRef,
capacity: usize,
) -> RecordBatchReceiverStreamBuilder {
RecordBatchReceiverStreamBuilder::new(schema, capacity)
}
}
pin_project! {
/// Combines a [`Stream`] with a [`SchemaRef`] implementing
/// [`RecordBatchStream`] for the combination
pub struct RecordBatchStreamAdapter<S> {
schema: SchemaRef,
#[pin]
stream: S,
}
}
impl<S> RecordBatchStreamAdapter<S> {
/// Creates a new [`RecordBatchStreamAdapter`] from the provided schema and stream
pub fn new(schema: SchemaRef, stream: S) -> Self {
Self { schema, stream }
}
}
impl<S> std::fmt::Debug for RecordBatchStreamAdapter<S> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("RecordBatchStreamAdapter")
.field("schema", &self.schema)
.finish()
}
}
impl<S> Stream for RecordBatchStreamAdapter<S>
where
S: Stream<Item = Result<RecordBatch>>,
{
type Item = Result<RecordBatch>;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
self.project().stream.poll_next(cx)
}
fn size_hint(&self) -> (usize, Option<usize>) {
self.stream.size_hint()
}
}
impl<S> RecordBatchStream for RecordBatchStreamAdapter<S>
where
S: Stream<Item = Result<RecordBatch>>,
{
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
}
/// `EmptyRecordBatchStream` can be used to create a [`RecordBatchStream`]
/// that will produce no results
pub struct EmptyRecordBatchStream {
/// Schema wrapped by Arc
schema: SchemaRef,
}
impl EmptyRecordBatchStream {
/// Create an empty RecordBatchStream
pub fn new(schema: SchemaRef) -> Self {
Self { schema }
}
}
impl RecordBatchStream for EmptyRecordBatchStream {
fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
}
impl Stream for EmptyRecordBatchStream {
type Item = Result<RecordBatch>;
fn poll_next(
self: Pin<&mut Self>,
_cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
Poll::Ready(None)
}
}
/// Stream wrapper that records `BaselineMetrics` for a particular
/// `[SendableRecordBatchStream]` (likely a partition)
pub(crate) struct ObservedStream {
inner: SendableRecordBatchStream,
baseline_metrics: BaselineMetrics,
}
impl ObservedStream {
pub fn new(
inner: SendableRecordBatchStream,
baseline_metrics: BaselineMetrics,
) -> Self {
Self {
inner,
baseline_metrics,
}
}
}
impl RecordBatchStream for ObservedStream {
fn schema(&self) -> SchemaRef {
self.inner.schema()
}
}
impl Stream for ObservedStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
let poll = self.inner.poll_next_unpin(cx);
self.baseline_metrics.record_poll(poll)
}
}
#[cfg(test)]
mod test {
use super::*;
use crate::test::exec::{
assert_strong_count_converges_to_zero, BlockingExec, MockExec, PanicExec,
};
use arrow_schema::{DataType, Field, Schema};
use datafusion_common::exec_err;
fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)]))
}
#[tokio::test]
#[should_panic(expected = "PanickingStream did panic")]
async fn record_batch_receiver_stream_propagates_panics() {
let schema = schema();
let num_partitions = 10;
let input = PanicExec::new(Arc::clone(&schema), num_partitions);
consume(input, 10).await
}
#[tokio::test]
#[should_panic(expected = "PanickingStream did panic: 1")]
async fn record_batch_receiver_stream_propagates_panics_early_shutdown() {
let schema = schema();
// Make 2 partitions, second partition panics before the first
let num_partitions = 2;
let input = PanicExec::new(Arc::clone(&schema), num_partitions)
.with_partition_panic(0, 10)
.with_partition_panic(1, 3); // partition 1 should panic first (after 3 )
// Ensure that the panic results in an early shutdown (that
// everything stops after the first panic).
// Since the stream reads every other batch: (0,1,0,1,0,panic)
// so should not exceed 5 batches prior to the panic
let max_batches = 5;
consume(input, max_batches).await
}
#[tokio::test]
async fn record_batch_receiver_stream_drop_cancel() {
let task_ctx = Arc::new(TaskContext::default());
let schema = schema();
// Make an input that never proceeds
let input = BlockingExec::new(Arc::clone(&schema), 1);
let refs = input.refs();
// Configure a RecordBatchReceiverStream to consume the input
let mut builder = RecordBatchReceiverStream::builder(schema, 2);
builder.run_input(Arc::new(input), 0, Arc::clone(&task_ctx));
let stream = builder.build();
// Input should still be present
assert!(std::sync::Weak::strong_count(&refs) > 0);
// Drop the stream, ensure the refs go to zero
drop(stream);
assert_strong_count_converges_to_zero(refs).await;
}
#[tokio::test]
/// Ensure that if an error is received in one stream, the
/// `RecordBatchReceiverStream` stops early and does not drive
/// other streams to completion.
async fn record_batch_receiver_stream_error_does_not_drive_completion() {
let task_ctx = Arc::new(TaskContext::default());
let schema = schema();
// make an input that will error twice
let error_stream = MockExec::new(
vec![exec_err!("Test1"), exec_err!("Test2")],
Arc::clone(&schema),
)
.with_use_task(false);
let mut builder = RecordBatchReceiverStream::builder(schema, 2);
builder.run_input(Arc::new(error_stream), 0, Arc::clone(&task_ctx));
let mut stream = builder.build();
// Get the first result, which should be an error
let first_batch = stream.next().await.unwrap();
let first_err = first_batch.unwrap_err();
assert_eq!(first_err.strip_backtrace(), "Execution error: Test1");
// There should be no more batches produced (should not get the second error)
assert!(stream.next().await.is_none());
}
/// Consumes all the input's partitions into a
/// RecordBatchReceiverStream and runs it to completion
///
/// panic's if more than max_batches is seen,
async fn consume(input: PanicExec, max_batches: usize) {
let task_ctx = Arc::new(TaskContext::default());
let input = Arc::new(input);
let num_partitions = input.properties().output_partitioning().partition_count();
// Configure a RecordBatchReceiverStream to consume all the input partitions
let mut builder =
RecordBatchReceiverStream::builder(input.schema(), num_partitions);
for partition in 0..num_partitions {
builder.run_input(
Arc::clone(&input) as Arc<dyn ExecutionPlan>,
partition,
Arc::clone(&task_ctx),
);
}
let mut stream = builder.build();
// Drain the stream until it is complete, panic'ing on error
let mut num_batches = 0;
while let Some(next) = stream.next().await {
next.unwrap();
num_batches += 1;
assert!(
num_batches < max_batches,
"Got the limit of {num_batches} batches before seeing panic"
);
}
}
}