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



##########
File path: datafusion/core/src/physical_plan/sorts/sort2.rs
##########
@@ -0,0 +1,1147 @@
+// 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.
+
+//! 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::context::TaskContext;
+use crate::execution::memory_manager::{
+    human_readable_size, ConsumerType, MemoryConsumer, MemoryConsumerId, 
MemoryManager,
+};
+use crate::execution::runtime_env::RuntimeEnv;
+use crate::physical_plan::common::{batch_byte_size, IPCWriter, 
SizedRecordBatchStream};
+use crate::physical_plan::expressions::PhysicalSortExpr;
+use crate::physical_plan::metrics::{
+    BaselineMetrics, CompositeMetricsSet, MemTrackingMetrics, MetricsSet,
+};
+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::{
+    DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, 
Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+use crate::prelude::SessionConfig;
+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::ipc::reader::FileReader;
+use arrow::record_batch::RecordBatch;
+use async_trait::async_trait;
+use futures::lock::Mutex;
+use futures::{Stream, StreamExt};
+use log::{debug, error};
+use std::any::Any;
+use std::cmp::min;
+use std::fmt;
+use std::fmt::{Debug, Formatter};
+use std::fs::File;
+use std::io::BufReader;
+use std::path::{Path, PathBuf};
+use std::sync::Arc;
+use std::task::{Context, Poll};
+use tempfile::NamedTempFile;
+use tokio::sync::mpsc::{Receiver, Sender};
+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 ExternalSorter2 {
+    id: MemoryConsumerId,
+    schema: SchemaRef,
+    in_mem_batches: Mutex<Vec<RecordBatch>>,
+    spills: Mutex<Vec<NamedTempFile>>,
+    /// Sort expressions
+    expr: Vec<PhysicalSortExpr>,
+    session_config: Arc<SessionConfig>,
+    runtime: Arc<RuntimeEnv>,
+    metrics_set: CompositeMetricsSet,
+    metrics: BaselineMetrics,
+}
+
+impl ExternalSorter2 {
+    pub fn new(
+        partition_id: usize,
+        schema: SchemaRef,
+        expr: Vec<PhysicalSortExpr>,
+        metrics_set: CompositeMetricsSet,
+        session_config: Arc<SessionConfig>,
+        runtime: Arc<RuntimeEnv>,
+    ) -> Self {
+        let metrics = metrics_set.new_intermediate_baseline(partition_id);
+        Self {
+            id: MemoryConsumerId::new(partition_id),
+            schema,
+            in_mem_batches: Mutex::new(vec![]),
+            spills: Mutex::new(vec![]),
+            expr,
+            session_config,
+            runtime,
+            metrics_set,
+            metrics,
+        }
+    }
+
+    async fn insert_batch(
+        &self,
+        input: RecordBatch,
+        tracking_metrics: &MemTrackingMetrics,
+    ) -> Result<()> {
+        if input.num_rows() > 0 {
+            let size = batch_byte_size(&input);
+            self.try_grow(size).await?;
+            self.metrics.mem_used().add(size);
+            let mut in_mem_batches = self.in_mem_batches.lock().await;
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            let partial = sort_batch(input, self.schema.clone(), &self.expr)?;

Review comment:
       The other change: sort each batch before buffering it in memory.




-- 
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