Skip to content
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
31 changes: 17 additions & 14 deletions datafusion/core/src/physical_plan/sorts/sort.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ use crate::physical_plan::metrics::{
};
use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeStream;
use crate::physical_plan::sorts::SortedStream;
use crate::physical_plan::stream::RecordBatchReceiverStream;
use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter};
use crate::physical_plan::{
DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning,
RecordBatchStream, SendableRecordBatchStream, Statistics,
Expand All @@ -42,12 +42,12 @@ use arrow::array::{make_array, Array, ArrayRef, MutableArrayData, UInt32Array};
pub use arrow::compute::SortOptions;
use arrow::compute::{concat, lexsort_to_indices, take, SortColumn, TakeOptions};
use arrow::datatypes::SchemaRef;
use arrow::error::Result as ArrowResult;
use arrow::error::{ArrowError, Result as ArrowResult};
use arrow::ipc::reader::FileReader;
use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use futures::lock::Mutex;
use futures::{Stream, StreamExt};
use futures::{Stream, StreamExt, TryFutureExt, TryStreamExt};
use log::{debug, error};
use std::any::Any;
use std::cmp::min;
Expand Down Expand Up @@ -779,17 +779,20 @@ impl ExecutionPlan for SortExec {

debug!("End SortExec's input.execute for partition: {}", partition);

let result = do_sort(
input,
partition,
self.expr.clone(),
self.metrics_set.clone(),
context,
)
.await;

debug!("End SortExec::execute for partition {}", partition);
result
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(
do_sort(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

A nicer fix would be to make ExternalSort stream, or less async, or some combination of the two, but in the short term this is a quick fix

input,
partition,
self.expr.clone(),
self.metrics_set.clone(),
context,
)
.map_err(|e| ArrowError::ExternalError(Box::new(e))),
)
.try_flatten(),
)))
}

fn metrics(&self) -> Option<MetricsSet> {
Expand Down
50 changes: 50 additions & 0 deletions datafusion/core/src/physical_plan/stream.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ use arrow::{
datatypes::SchemaRef, error::Result as ArrowResult, record_batch::RecordBatch,
};
use futures::{Stream, StreamExt};
use pin_project_lite::pin_project;
use tokio::task::JoinHandle;
use tokio_stream::wrappers::ReceiverStream;

Expand Down Expand Up @@ -73,3 +74,52 @@ impl RecordBatchStream for RecordBatchReceiverStream {
self.schema.clone()
}
}

pin_project! {
/// Combines a [`Stream`] with a [`SchemaRef`] implementing
/// [`RecordBatchStream`] for the combination
pub(crate) struct RecordBatchStreamAdapter<S> {
Copy link
Contributor

Choose a reason for hiding this comment

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

👍

schema: SchemaRef,

#[pin]
stream: S,
}
}

impl<S> RecordBatchStreamAdapter<S> {
/// Creates a new [`RecordBatchStreamAdapter`] from the provided schema and stream
pub(crate) 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 = ArrowResult<RecordBatch>>,
{
type Item = ArrowResult<RecordBatch>;

fn poll_next(
self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> std::task::Poll<Option<Self::Item>> {
self.project().stream.poll_next(cx)
}
}

impl<S> RecordBatchStream for RecordBatchStreamAdapter<S>
where
S: Stream<Item = ArrowResult<RecordBatch>>,
{
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}