yjshen commented on a change in pull request #1596:
URL: https://github.com/apache/arrow-datafusion/pull/1596#discussion_r788335836



##########
File path: datafusion/src/physical_plan/sorts/sort.rs
##########
@@ -15,47 +15,432 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//! Defines the SORT plan
+//! Sort that deals with an arbitrary size of the input.
+//! It will do in-memory sorting if it has enough memory budget
+//! but spills to disk if needed.
 
 use crate::error::{DataFusionError, Result};
+use crate::execution::memory_manager::{
+    ConsumerType, MemoryConsumer, MemoryConsumerId, MemoryManager,
+};
 use crate::execution::runtime_env::RuntimeEnv;
-use crate::physical_plan::common::AbortOnDropSingle;
+use crate::physical_plan::common::{batch_byte_size, IPCWriter, 
SizedRecordBatchStream};
 use crate::physical_plan::expressions::PhysicalSortExpr;
 use crate::physical_plan::metrics::{
-    BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, RecordOutput,
+    BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricsSet, Time,
 };
+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::{
-    common, DisplayFormatType, Distribution, ExecutionPlan, Partitioning,
+    common, DisplayFormatType, Distribution, EmptyRecordBatchStream, 
ExecutionPlan,
+    Partitioning, SendableRecordBatchStream, Statistics,
 };
-use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream, 
Statistics};
+use arrow::array::ArrayRef;
 pub use arrow::compute::SortOptions;
 use arrow::compute::{lexsort_to_indices, take, SortColumn, TakeOptions};
 use arrow::datatypes::SchemaRef;
 use arrow::error::Result as ArrowResult;
+use arrow::ipc::reader::FileReader;
 use arrow::record_batch::RecordBatch;
-use arrow::{array::ArrayRef, error::ArrowError};
 use async_trait::async_trait;
-use futures::stream::Stream;
-use futures::Future;
-use pin_project_lite::pin_project;
+use futures::lock::Mutex;
+use futures::StreamExt;
+use log::{error, info};
 use std::any::Any;
-use std::pin::Pin;
+use std::fmt;
+use std::fmt::{Debug, Formatter};
+use std::fs::File;
+use std::io::BufReader;
+use std::sync::atomic::{AtomicUsize, Ordering};
 use std::sync::Arc;
-use std::task::{Context, Poll};
+use std::time::Duration;
+use tokio::sync::mpsc::{Receiver as TKReceiver, Sender as TKSender};
+use tokio::task;
+
+/// Sort arbitrary size of data to get a total order (may spill several times 
during sorting based on free memory available).
+///
+/// The basic architecture of the algorithm:
+/// 1. get a non-empty new batch from input
+/// 2. check with the memory manager if we could buffer the batch in memory
+/// 2.1 if memory sufficient, then buffer batch in memory, go to 1.
+/// 2.2 if the memory threshold is reached, sort all buffered batches and 
spill to file.
+///     buffer the batch in memory, go to 1.
+/// 3. when input is exhausted, merge all in memory batches and spills to get 
a total order.
+struct ExternalSorter {
+    id: MemoryConsumerId,
+    schema: SchemaRef,
+    in_mem_batches: Mutex<Vec<RecordBatch>>,
+    spills: Mutex<Vec<String>>,
+    /// Sort expressions
+    expr: Vec<PhysicalSortExpr>,
+    runtime: Arc<RuntimeEnv>,
+    metrics: AggregatedMetricsSet,
+    used: AtomicUsize,
+    spilled_bytes: AtomicUsize,
+    spilled_count: AtomicUsize,
+}
 
-/// Sort execution plan
+impl ExternalSorter {
+    pub fn new(
+        partition_id: usize,
+        schema: SchemaRef,
+        expr: Vec<PhysicalSortExpr>,
+        metrics: AggregatedMetricsSet,
+        runtime: Arc<RuntimeEnv>,
+    ) -> Self {
+        Self {
+            id: MemoryConsumerId::new(partition_id),
+            schema,
+            in_mem_batches: Mutex::new(vec![]),
+            spills: Mutex::new(vec![]),
+            expr,
+            runtime,
+            metrics,
+            used: AtomicUsize::new(0),
+            spilled_bytes: AtomicUsize::new(0),
+            spilled_count: AtomicUsize::new(0),
+        }
+    }
+
+    async fn insert_batch(&self, input: RecordBatch) -> Result<()> {
+        if input.num_rows() > 0 {
+            let size = batch_byte_size(&input);
+            self.try_grow(size).await?;
+            self.used.fetch_add(size, Ordering::SeqCst);
+            let mut in_mem_batches = self.in_mem_batches.lock().await;
+            in_mem_batches.push(input);
+        }
+        Ok(())
+    }
+
+    async fn spilled_before(&self) -> bool {
+        let spills = self.spills.lock().await;
+        !spills.is_empty()
+    }
+
+    /// MergeSort in mem batches as well as spills into total order with 
`SortPreservingMergeStream`.
+    async fn sort(&self) -> Result<SendableRecordBatchStream> {

Review comment:
       Sorry I cannot quite follow here. 🤔  I think although async, rust would 
compile `do_sort` into a state machine, run as the sequence in it?
   
   ```rust
   async fn do_sort(
       mut input: SendableRecordBatchStream,
       partition_id: usize,
       expr: Vec<PhysicalSortExpr>,
       metrics: AggregatedMetricsSet,
       runtime: Arc<RuntimeEnv>,
   ) -> Result<SendableRecordBatchStream> {
       let schema = input.schema();
       let sorter = Arc::new(ExternalSorter::new(
           partition_id,
           schema.clone(),
           expr,
           metrics,
           runtime.clone(),
       ));
       runtime.register_consumer(&(sorter.clone() as Arc<dyn MemoryConsumer>));
   
       while let Some(batch) = input.next().await {
           let batch = batch?;
           sorter.insert_batch(batch).await?;
       }
   
       let result = sorter.sort().await;
       runtime.drop_consumer(sorter.id());
       result
   }
   ```
   
   Is that possible sort executed before `insert_batch()` in the while let? How 
does `concurrently` affect this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to