You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/01 11:33:51 UTC

[GitHub] [arrow-datafusion] yjshen opened a new pull request #2132: WIP: Reduce sort memory usage v1

yjshen opened a new pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132


   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #.
   
    # Rationale for this change
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   # What changes are included in this PR?
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are there any user-facing changes?
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085799873


   A modified version of TPC-H q1:
   
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   
   Before this PR:
   
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   
         
   With this PR:
                                                                              
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
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:
       Another 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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen removed a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen removed a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085799873


   A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   ```shell
   ./target/release/tpch benchmark datafusion --path /Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet -f parquet -q 1 
   ```
   
   Before this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   ```
   
         
   With this PR:
                                          
   ```                                    
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   ```
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085806095


   A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   ```shell
   ./target/release/tpch benchmark datafusion --path /Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet -f parquet -q 1 
   ```
   
   Before this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   ```
   
         
   With this PR:
                                          
   ```                                    
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   ```
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r840496480



##########
File path: benchmarks/queries/q1.sql
##########
@@ -1,21 +1,12 @@
 select
     l_returnflag,
     l_linestatus,
-    sum(l_quantity) as sum_qty,
-    sum(l_extendedprice) as sum_base_price,
-    sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
-    sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
-    avg(l_quantity) as avg_qty,
-    avg(l_extendedprice) as avg_price,
-    avg(l_discount) as avg_disc,
-    count(*) as count_order
+    l_quantity,
+    l_extendedprice,
+    l_discount,
+    l_tax
 from
     lineitem
-where

Review comment:
       tmp change to bench sort

##########
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)?;
+            in_mem_batches.push(partial);
+        }
+        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> {
+        let partition = self.partition_id();
+        let batch_size = self.session_config.batch_size;
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+
+        if self.spilled_before().await {
+            let tracking_metrics = self
+                .metrics_set
+                .new_intermediate_tracking(partition, self.runtime.clone());
+            let mut streams: Vec<SortedStream> = vec![];
+            if in_mem_batches.len() > 0 {
+                let in_mem_stream = in_mem_partial_sort(
+                    &mut *in_mem_batches,
+                    self.schema.clone(),
+                    &self.expr,
+                    batch_size,
+                    tracking_metrics,
+                )?;
+                let prev_used = self.metrics.mem_used().set(0);
+                streams.push(SortedStream::new(in_mem_stream, prev_used));
+            }
+
+            let mut spills = self.spills.lock().await;
+
+            for spill in spills.drain(..) {
+                let stream = read_spill_as_stream(spill, self.schema.clone())?;
+                streams.push(SortedStream::new(stream, 0));
+            }
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            Ok(Box::pin(SortPreservingMergeStream::new_from_streams(
+                streams,
+                self.schema.clone(),
+                &self.expr,
+                tracking_metrics,
+                self.session_config.batch_size,
+            )))
+        } else if in_mem_batches.len() > 0 {
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            let result = in_mem_partial_sort(
+                &mut *in_mem_batches,
+                self.schema.clone(),
+                &self.expr,
+                batch_size,
+                tracking_metrics,
+            );
+            // Report to the memory manager we are no longer using memory
+            self.metrics.mem_used().set(0);
+            result
+        } else {
+            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+        }
+    }
+
+    fn used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+
+    fn spilled_bytes(&self) -> usize {
+        self.metrics.spilled_bytes().value()
+    }
+
+    fn spill_count(&self) -> usize {
+        self.metrics.spill_count().value()
+    }
+}
+
+impl Debug for ExternalSorter2 {
+    fn fmt(&self, f: &mut Formatter) -> fmt::Result {
+        f.debug_struct("ExternalSorter2")
+            .field("id", &self.id())
+            .field("memory_used", &self.used())
+            .field("spilled_bytes", &self.spilled_bytes())
+            .field("spill_count", &self.spill_count())
+            .finish()
+    }
+}
+
+#[derive(Debug, Copy, Clone)]
+struct CombinedIndex {
+    batch_idx: usize,
+    row_idx: usize,
+}
+
+impl Drop for ExternalSorter2 {
+    fn drop(&mut self) {
+        self.runtime.drop_consumer(self.id(), self.used());
+    }
+}
+
+#[async_trait]
+impl MemoryConsumer for ExternalSorter2 {
+    fn name(&self) -> String {
+        "ExternalSorter2".to_owned()
+    }
+
+    fn id(&self) -> &MemoryConsumerId {
+        &self.id
+    }
+
+    fn memory_manager(&self) -> Arc<MemoryManager> {
+        self.runtime.memory_manager.clone()
+    }
+
+    fn type_(&self) -> &ConsumerType {
+        &ConsumerType::Requesting
+    }
+
+    async fn spill(&self) -> Result<usize> {
+        debug!(
+            "{}[{}] spilling sort data of {} to disk while inserting ({} time(s) so far)",
+            self.name(),
+            self.id(),
+            self.used(),
+            self.spill_count()
+        );
+
+        let partition = self.partition_id();
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+        // we could always get a chance to free some memory as long as we are holding some
+        if in_mem_batches.len() == 0 {
+            return Ok(0);
+        }
+
+        let tracking_metrics = self
+            .metrics_set
+            .new_intermediate_tracking(partition, self.runtime.clone());
+
+        let spillfile = self.runtime.disk_manager.create_tmp_file()?;
+        let stream = in_mem_partial_sort(
+            &mut *in_mem_batches,
+            self.schema.clone(),
+            &*self.expr,
+            self.session_config.batch_size,
+            tracking_metrics,
+        );
+
+        spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone())
+            .await?;
+        let mut spills = self.spills.lock().await;
+        let used = self.metrics.mem_used().set(0);
+        self.metrics.record_spill(used);
+        spills.push(spillfile);
+        Ok(used)
+    }
+
+    fn mem_used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+}
+
+/// consume the non-empty `sorted_bathes` and do in_mem_sort
+fn in_mem_partial_sort(
+    buffered_batches: &mut Vec<RecordBatch>,
+    schema: SchemaRef,
+    expressions: &[PhysicalSortExpr],
+    batch_size: usize,
+    tracking_metrics: MemTrackingMetrics,
+) -> Result<SendableRecordBatchStream> {
+    assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
+
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns
   2. sort to get the index array
   3. use `CombinedIndex` to avoid huge batch construction
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   On my Linux Desktop:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ## A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.
   
   ## cargo criterion --bench sort_limit_query_sql
   
   Without this PR:
   ```
   sort_and_limit_by_int   time:   [869.27 us 870.72 us 872.19 us]                                  
   
   sort_and_limit_by_float time:   [845.95 us 847.36 us 848.77 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [875.07 us 876.80 us 878.59 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [878.78 us 880.40 us 882.06 us]
   
   ```
   
   With this PR:
   
   ```
   sort_and_limit_by_int   time:   [856.71 us 858.58 us 860.41 us]                                  
   
   sort_and_limit_by_float time:   [836.55 us 838.40 us 840.29 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [864.83 us 866.33 us 867.83 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [866.89 us 868.25 us 869.62 us]
   
   ```
   
   Similar performance as well.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r840501793



##########
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)?;
+            in_mem_batches.push(partial);
+        }
+        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> {
+        let partition = self.partition_id();
+        let batch_size = self.session_config.batch_size;
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+
+        if self.spilled_before().await {
+            let tracking_metrics = self
+                .metrics_set
+                .new_intermediate_tracking(partition, self.runtime.clone());
+            let mut streams: Vec<SortedStream> = vec![];
+            if in_mem_batches.len() > 0 {
+                let in_mem_stream = in_mem_partial_sort(
+                    &mut *in_mem_batches,
+                    self.schema.clone(),
+                    &self.expr,
+                    batch_size,
+                    tracking_metrics,
+                )?;
+                let prev_used = self.metrics.mem_used().set(0);
+                streams.push(SortedStream::new(in_mem_stream, prev_used));
+            }
+
+            let mut spills = self.spills.lock().await;
+
+            for spill in spills.drain(..) {
+                let stream = read_spill_as_stream(spill, self.schema.clone())?;
+                streams.push(SortedStream::new(stream, 0));
+            }
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            Ok(Box::pin(SortPreservingMergeStream::new_from_streams(
+                streams,
+                self.schema.clone(),
+                &self.expr,
+                tracking_metrics,
+                self.session_config.batch_size,
+            )))
+        } else if in_mem_batches.len() > 0 {
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            let result = in_mem_partial_sort(
+                &mut *in_mem_batches,
+                self.schema.clone(),
+                &self.expr,
+                batch_size,
+                tracking_metrics,
+            );
+            // Report to the memory manager we are no longer using memory
+            self.metrics.mem_used().set(0);
+            result
+        } else {
+            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+        }
+    }
+
+    fn used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+
+    fn spilled_bytes(&self) -> usize {
+        self.metrics.spilled_bytes().value()
+    }
+
+    fn spill_count(&self) -> usize {
+        self.metrics.spill_count().value()
+    }
+}
+
+impl Debug for ExternalSorter2 {
+    fn fmt(&self, f: &mut Formatter) -> fmt::Result {
+        f.debug_struct("ExternalSorter2")
+            .field("id", &self.id())
+            .field("memory_used", &self.used())
+            .field("spilled_bytes", &self.spilled_bytes())
+            .field("spill_count", &self.spill_count())
+            .finish()
+    }
+}
+
+#[derive(Debug, Copy, Clone)]
+struct CombinedIndex {
+    batch_idx: usize,
+    row_idx: usize,
+}
+
+impl Drop for ExternalSorter2 {
+    fn drop(&mut self) {
+        self.runtime.drop_consumer(self.id(), self.used());
+    }
+}
+
+#[async_trait]
+impl MemoryConsumer for ExternalSorter2 {
+    fn name(&self) -> String {
+        "ExternalSorter2".to_owned()
+    }
+
+    fn id(&self) -> &MemoryConsumerId {
+        &self.id
+    }
+
+    fn memory_manager(&self) -> Arc<MemoryManager> {
+        self.runtime.memory_manager.clone()
+    }
+
+    fn type_(&self) -> &ConsumerType {
+        &ConsumerType::Requesting
+    }
+
+    async fn spill(&self) -> Result<usize> {
+        debug!(
+            "{}[{}] spilling sort data of {} to disk while inserting ({} time(s) so far)",
+            self.name(),
+            self.id(),
+            self.used(),
+            self.spill_count()
+        );
+
+        let partition = self.partition_id();
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+        // we could always get a chance to free some memory as long as we are holding some
+        if in_mem_batches.len() == 0 {
+            return Ok(0);
+        }
+
+        let tracking_metrics = self
+            .metrics_set
+            .new_intermediate_tracking(partition, self.runtime.clone());
+
+        let spillfile = self.runtime.disk_manager.create_tmp_file()?;
+        let stream = in_mem_partial_sort(
+            &mut *in_mem_batches,
+            self.schema.clone(),
+            &*self.expr,
+            self.session_config.batch_size,
+            tracking_metrics,
+        );
+
+        spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone())
+            .await?;
+        let mut spills = self.spills.lock().await;
+        let used = self.metrics.mem_used().set(0);
+        self.metrics.record_spill(used);
+        spills.push(spillfile);
+        Ok(used)
+    }
+
+    fn mem_used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+}
+
+/// consume the non-empty `sorted_bathes` and do in_mem_sort
+fn in_mem_partial_sort(
+    buffered_batches: &mut Vec<RecordBatch>,
+    schema: SchemaRef,
+    expressions: &[PhysicalSortExpr],
+    batch_size: usize,
+    tracking_metrics: MemTrackingMetrics,
+) -> Result<SendableRecordBatchStream> {
+    assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
+
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns (instead of all columns)
   2. sort to get the index array (same with original sort)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r840501793



##########
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)?;
+            in_mem_batches.push(partial);
+        }
+        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> {
+        let partition = self.partition_id();
+        let batch_size = self.session_config.batch_size;
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+
+        if self.spilled_before().await {
+            let tracking_metrics = self
+                .metrics_set
+                .new_intermediate_tracking(partition, self.runtime.clone());
+            let mut streams: Vec<SortedStream> = vec![];
+            if in_mem_batches.len() > 0 {
+                let in_mem_stream = in_mem_partial_sort(
+                    &mut *in_mem_batches,
+                    self.schema.clone(),
+                    &self.expr,
+                    batch_size,
+                    tracking_metrics,
+                )?;
+                let prev_used = self.metrics.mem_used().set(0);
+                streams.push(SortedStream::new(in_mem_stream, prev_used));
+            }
+
+            let mut spills = self.spills.lock().await;
+
+            for spill in spills.drain(..) {
+                let stream = read_spill_as_stream(spill, self.schema.clone())?;
+                streams.push(SortedStream::new(stream, 0));
+            }
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            Ok(Box::pin(SortPreservingMergeStream::new_from_streams(
+                streams,
+                self.schema.clone(),
+                &self.expr,
+                tracking_metrics,
+                self.session_config.batch_size,
+            )))
+        } else if in_mem_batches.len() > 0 {
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            let result = in_mem_partial_sort(
+                &mut *in_mem_batches,
+                self.schema.clone(),
+                &self.expr,
+                batch_size,
+                tracking_metrics,
+            );
+            // Report to the memory manager we are no longer using memory
+            self.metrics.mem_used().set(0);
+            result
+        } else {
+            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+        }
+    }
+
+    fn used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+
+    fn spilled_bytes(&self) -> usize {
+        self.metrics.spilled_bytes().value()
+    }
+
+    fn spill_count(&self) -> usize {
+        self.metrics.spill_count().value()
+    }
+}
+
+impl Debug for ExternalSorter2 {
+    fn fmt(&self, f: &mut Formatter) -> fmt::Result {
+        f.debug_struct("ExternalSorter2")
+            .field("id", &self.id())
+            .field("memory_used", &self.used())
+            .field("spilled_bytes", &self.spilled_bytes())
+            .field("spill_count", &self.spill_count())
+            .finish()
+    }
+}
+
+#[derive(Debug, Copy, Clone)]
+struct CombinedIndex {
+    batch_idx: usize,
+    row_idx: usize,
+}
+
+impl Drop for ExternalSorter2 {
+    fn drop(&mut self) {
+        self.runtime.drop_consumer(self.id(), self.used());
+    }
+}
+
+#[async_trait]
+impl MemoryConsumer for ExternalSorter2 {
+    fn name(&self) -> String {
+        "ExternalSorter2".to_owned()
+    }
+
+    fn id(&self) -> &MemoryConsumerId {
+        &self.id
+    }
+
+    fn memory_manager(&self) -> Arc<MemoryManager> {
+        self.runtime.memory_manager.clone()
+    }
+
+    fn type_(&self) -> &ConsumerType {
+        &ConsumerType::Requesting
+    }
+
+    async fn spill(&self) -> Result<usize> {
+        debug!(
+            "{}[{}] spilling sort data of {} to disk while inserting ({} time(s) so far)",
+            self.name(),
+            self.id(),
+            self.used(),
+            self.spill_count()
+        );
+
+        let partition = self.partition_id();
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+        // we could always get a chance to free some memory as long as we are holding some
+        if in_mem_batches.len() == 0 {
+            return Ok(0);
+        }
+
+        let tracking_metrics = self
+            .metrics_set
+            .new_intermediate_tracking(partition, self.runtime.clone());
+
+        let spillfile = self.runtime.disk_manager.create_tmp_file()?;
+        let stream = in_mem_partial_sort(
+            &mut *in_mem_batches,
+            self.schema.clone(),
+            &*self.expr,
+            self.session_config.batch_size,
+            tracking_metrics,
+        );
+
+        spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone())
+            .await?;
+        let mut spills = self.spills.lock().await;
+        let used = self.metrics.mem_used().set(0);
+        self.metrics.record_spill(used);
+        spills.push(spillfile);
+        Ok(used)
+    }
+
+    fn mem_used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+}
+
+/// consume the non-empty `sorted_bathes` and do in_mem_sort
+fn in_mem_partial_sort(
+    buffered_batches: &mut Vec<RecordBatch>,
+    schema: SchemaRef,
+    expressions: &[PhysicalSortExpr],
+    batch_size: usize,
+    tracking_metrics: MemTrackingMetrics,
+) -> Result<SendableRecordBatchStream> {
+    assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
+
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns (instead of all columns)
   2. sort to get the index array (same as the original sort)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   
   On a newer Linux Desktop:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r840501793



##########
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)?;
+            in_mem_batches.push(partial);
+        }
+        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> {
+        let partition = self.partition_id();
+        let batch_size = self.session_config.batch_size;
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+
+        if self.spilled_before().await {
+            let tracking_metrics = self
+                .metrics_set
+                .new_intermediate_tracking(partition, self.runtime.clone());
+            let mut streams: Vec<SortedStream> = vec![];
+            if in_mem_batches.len() > 0 {
+                let in_mem_stream = in_mem_partial_sort(
+                    &mut *in_mem_batches,
+                    self.schema.clone(),
+                    &self.expr,
+                    batch_size,
+                    tracking_metrics,
+                )?;
+                let prev_used = self.metrics.mem_used().set(0);
+                streams.push(SortedStream::new(in_mem_stream, prev_used));
+            }
+
+            let mut spills = self.spills.lock().await;
+
+            for spill in spills.drain(..) {
+                let stream = read_spill_as_stream(spill, self.schema.clone())?;
+                streams.push(SortedStream::new(stream, 0));
+            }
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            Ok(Box::pin(SortPreservingMergeStream::new_from_streams(
+                streams,
+                self.schema.clone(),
+                &self.expr,
+                tracking_metrics,
+                self.session_config.batch_size,
+            )))
+        } else if in_mem_batches.len() > 0 {
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            let result = in_mem_partial_sort(
+                &mut *in_mem_batches,
+                self.schema.clone(),
+                &self.expr,
+                batch_size,
+                tracking_metrics,
+            );
+            // Report to the memory manager we are no longer using memory
+            self.metrics.mem_used().set(0);
+            result
+        } else {
+            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+        }
+    }
+
+    fn used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+
+    fn spilled_bytes(&self) -> usize {
+        self.metrics.spilled_bytes().value()
+    }
+
+    fn spill_count(&self) -> usize {
+        self.metrics.spill_count().value()
+    }
+}
+
+impl Debug for ExternalSorter2 {
+    fn fmt(&self, f: &mut Formatter) -> fmt::Result {
+        f.debug_struct("ExternalSorter2")
+            .field("id", &self.id())
+            .field("memory_used", &self.used())
+            .field("spilled_bytes", &self.spilled_bytes())
+            .field("spill_count", &self.spill_count())
+            .finish()
+    }
+}
+
+#[derive(Debug, Copy, Clone)]
+struct CombinedIndex {
+    batch_idx: usize,
+    row_idx: usize,
+}
+
+impl Drop for ExternalSorter2 {
+    fn drop(&mut self) {
+        self.runtime.drop_consumer(self.id(), self.used());
+    }
+}
+
+#[async_trait]
+impl MemoryConsumer for ExternalSorter2 {
+    fn name(&self) -> String {
+        "ExternalSorter2".to_owned()
+    }
+
+    fn id(&self) -> &MemoryConsumerId {
+        &self.id
+    }
+
+    fn memory_manager(&self) -> Arc<MemoryManager> {
+        self.runtime.memory_manager.clone()
+    }
+
+    fn type_(&self) -> &ConsumerType {
+        &ConsumerType::Requesting
+    }
+
+    async fn spill(&self) -> Result<usize> {
+        debug!(
+            "{}[{}] spilling sort data of {} to disk while inserting ({} time(s) so far)",
+            self.name(),
+            self.id(),
+            self.used(),
+            self.spill_count()
+        );
+
+        let partition = self.partition_id();
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+        // we could always get a chance to free some memory as long as we are holding some
+        if in_mem_batches.len() == 0 {
+            return Ok(0);
+        }
+
+        let tracking_metrics = self
+            .metrics_set
+            .new_intermediate_tracking(partition, self.runtime.clone());
+
+        let spillfile = self.runtime.disk_manager.create_tmp_file()?;
+        let stream = in_mem_partial_sort(
+            &mut *in_mem_batches,
+            self.schema.clone(),
+            &*self.expr,
+            self.session_config.batch_size,
+            tracking_metrics,
+        );
+
+        spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone())
+            .await?;
+        let mut spills = self.spills.lock().await;
+        let used = self.metrics.mem_used().set(0);
+        self.metrics.record_spill(used);
+        spills.push(spillfile);
+        Ok(used)
+    }
+
+    fn mem_used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+}
+
+/// consume the non-empty `sorted_bathes` and do in_mem_sort
+fn in_mem_partial_sort(
+    buffered_batches: &mut Vec<RecordBatch>,
+    schema: SchemaRef,
+    expressions: &[PhysicalSortExpr],
+    batch_size: usize,
+    tracking_metrics: MemTrackingMetrics,
+) -> Result<SendableRecordBatchStream> {
+    assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
+
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns (instead of all columns)
   2. sort to get the index array (same)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r841011038



##########
File path: datafusion/core/src/physical_plan/sorts/sort.rs
##########
@@ -271,33 +291,212 @@ fn in_mem_partial_sort(
     buffered_batches: &mut Vec<RecordBatch>,
     schema: SchemaRef,
     expressions: &[PhysicalSortExpr],
+    batch_size: usize,
     tracking_metrics: MemTrackingMetrics,
 ) -> Result<SendableRecordBatchStream> {
     assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
 
-    let result = {
-        // NB timer records time taken on drop, so there are no
-        // calls to `timer.done()` below.
-        let _timer = tracking_metrics.elapsed_compute().timer();
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns (instead of all columns)
   2. sort to get the index array (same as the original sort)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.

##########
File path: datafusion/core/src/physical_plan/sorts/sort.rs
##########
@@ -105,13 +107,21 @@ impl ExternalSorter {
         }
     }
 
-    async fn insert_batch(&self, input: RecordBatch) -> Result<()> {
+    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;
-            in_mem_batches.push(input);
+            // 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.

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

##########
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)?;
+            in_mem_batches.push(partial);
+        }
+        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> {
+        let partition = self.partition_id();
+        let batch_size = self.session_config.batch_size;
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+
+        if self.spilled_before().await {
+            let tracking_metrics = self
+                .metrics_set
+                .new_intermediate_tracking(partition, self.runtime.clone());
+            let mut streams: Vec<SortedStream> = vec![];
+            if in_mem_batches.len() > 0 {
+                let in_mem_stream = in_mem_partial_sort(
+                    &mut *in_mem_batches,
+                    self.schema.clone(),
+                    &self.expr,
+                    batch_size,
+                    tracking_metrics,
+                )?;
+                let prev_used = self.metrics.mem_used().set(0);
+                streams.push(SortedStream::new(in_mem_stream, prev_used));
+            }
+
+            let mut spills = self.spills.lock().await;
+
+            for spill in spills.drain(..) {
+                let stream = read_spill_as_stream(spill, self.schema.clone())?;
+                streams.push(SortedStream::new(stream, 0));
+            }
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            Ok(Box::pin(SortPreservingMergeStream::new_from_streams(
+                streams,
+                self.schema.clone(),
+                &self.expr,
+                tracking_metrics,
+                self.session_config.batch_size,
+            )))
+        } else if in_mem_batches.len() > 0 {
+            let tracking_metrics = self
+                .metrics_set
+                .new_final_tracking(partition, self.runtime.clone());
+            let result = in_mem_partial_sort(
+                &mut *in_mem_batches,
+                self.schema.clone(),
+                &self.expr,
+                batch_size,
+                tracking_metrics,
+            );
+            // Report to the memory manager we are no longer using memory
+            self.metrics.mem_used().set(0);
+            result
+        } else {
+            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+        }
+    }
+
+    fn used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+
+    fn spilled_bytes(&self) -> usize {
+        self.metrics.spilled_bytes().value()
+    }
+
+    fn spill_count(&self) -> usize {
+        self.metrics.spill_count().value()
+    }
+}
+
+impl Debug for ExternalSorter2 {
+    fn fmt(&self, f: &mut Formatter) -> fmt::Result {
+        f.debug_struct("ExternalSorter2")
+            .field("id", &self.id())
+            .field("memory_used", &self.used())
+            .field("spilled_bytes", &self.spilled_bytes())
+            .field("spill_count", &self.spill_count())
+            .finish()
+    }
+}
+
+#[derive(Debug, Copy, Clone)]
+struct CombinedIndex {
+    batch_idx: usize,
+    row_idx: usize,
+}
+
+impl Drop for ExternalSorter2 {
+    fn drop(&mut self) {
+        self.runtime.drop_consumer(self.id(), self.used());
+    }
+}
+
+#[async_trait]
+impl MemoryConsumer for ExternalSorter2 {
+    fn name(&self) -> String {
+        "ExternalSorter2".to_owned()
+    }
+
+    fn id(&self) -> &MemoryConsumerId {
+        &self.id
+    }
+
+    fn memory_manager(&self) -> Arc<MemoryManager> {
+        self.runtime.memory_manager.clone()
+    }
+
+    fn type_(&self) -> &ConsumerType {
+        &ConsumerType::Requesting
+    }
+
+    async fn spill(&self) -> Result<usize> {
+        debug!(
+            "{}[{}] spilling sort data of {} to disk while inserting ({} time(s) so far)",
+            self.name(),
+            self.id(),
+            self.used(),
+            self.spill_count()
+        );
+
+        let partition = self.partition_id();
+        let mut in_mem_batches = self.in_mem_batches.lock().await;
+        // we could always get a chance to free some memory as long as we are holding some
+        if in_mem_batches.len() == 0 {
+            return Ok(0);
+        }
+
+        let tracking_metrics = self
+            .metrics_set
+            .new_intermediate_tracking(partition, self.runtime.clone());
+
+        let spillfile = self.runtime.disk_manager.create_tmp_file()?;
+        let stream = in_mem_partial_sort(
+            &mut *in_mem_batches,
+            self.schema.clone(),
+            &*self.expr,
+            self.session_config.batch_size,
+            tracking_metrics,
+        );
+
+        spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone())
+            .await?;
+        let mut spills = self.spills.lock().await;
+        let used = self.metrics.mem_used().set(0);
+        self.metrics.record_spill(used);
+        spills.push(spillfile);
+        Ok(used)
+    }
+
+    fn mem_used(&self) -> usize {
+        self.metrics.mem_used().value()
+    }
+}
+
+/// consume the non-empty `sorted_bathes` and do in_mem_sort
+fn in_mem_partial_sort(
+    buffered_batches: &mut Vec<RecordBatch>,
+    schema: SchemaRef,
+    expressions: &[PhysicalSortExpr],
+    batch_size: usize,
+    tracking_metrics: MemTrackingMetrics,
+) -> Result<SendableRecordBatchStream> {
+    assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
+
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concrete all sort-columns (instead of all columns)
   2. sort to get the index array (same as the original sort)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.

##########
File path: benchmarks/queries/q1.sql
##########
@@ -1,21 +1,12 @@
 select
     l_returnflag,
     l_linestatus,
-    sum(l_quantity) as sum_qty,
-    sum(l_extendedprice) as sum_base_price,
-    sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
-    sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
-    avg(l_quantity) as avg_qty,
-    avg(l_extendedprice) as avg_price,
-    avg(l_discount) as avg_disc,
-    count(*) as count_order
+    l_quantity,
+    l_extendedprice,
+    l_discount,
+    l_tax
 from
     lineitem
-where

Review comment:
       tmp change to bench sort




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085799873


   A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   ```shell
   ./target/release/tpch benchmark datafusion --path /Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet -f parquet -q 1 
   ```
   
   Before this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   ```
   
         
   With this PR:
                                          
   ```                                    
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   ```
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r841011135



##########
File path: datafusion/core/src/physical_plan/sorts/sort.rs
##########
@@ -105,13 +107,21 @@ impl ExternalSorter {
         }
     }
 
-    async fn insert_batch(&self, input: RecordBatch) -> Result<()> {
+    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;
-            in_mem_batches.push(input);
+            // 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:
       A change here: 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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r841011038



##########
File path: datafusion/core/src/physical_plan/sorts/sort.rs
##########
@@ -271,33 +291,212 @@ fn in_mem_partial_sort(
     buffered_batches: &mut Vec<RecordBatch>,
     schema: SchemaRef,
     expressions: &[PhysicalSortExpr],
+    batch_size: usize,
     tracking_metrics: MemTrackingMetrics,
 ) -> Result<SendableRecordBatchStream> {
     assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
 
-    let result = {
-        // NB timer records time taken on drop, so there are no
-        // calls to `timer.done()` below.
-        let _timer = tracking_metrics.elapsed_compute().timer();
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concat all sort-columns (instead of all columns)
   2. sort to get the index array (same as the original sort)
   3. use `CompositeIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen removed a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen removed a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085806095


   A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   ```shell
   ./target/release/tpch benchmark datafusion --path /Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet -f parquet -q 1 
   ```
   
   Before this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   ```
   
         
   With this PR:
                                          
   ```                                    
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   ```
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085799873


   A modified version of TPC-H q1:
   
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   
   ```shell
   ./target/release/tpch benchmark datafusion --path /Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet -f parquet -q 1 
   ```
   
   Before this PR:
   
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 5922.3 ms and returned 6001214 rows
   Query 1 iteration 1 took 5699.3 ms and returned 6001214 rows
   Query 1 iteration 2 took 6166.0 ms and returned 6001214 rows
   Query 1 avg time: 5929.20 ms
   
         
   With this PR:
                                                                              
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 8192, path: "/Users/shenyijie/code/arrow-datafusion/benchmarks/tpch-parquet", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 4031.2 ms and returned 6001214 rows
   Query 1 iteration 1 took 4375.6 ms and returned 6001214 rows
   Query 1 iteration 2 took 4318.8 ms and returned 6001214 rows
   Query 1 avg time: 4241.89 ms
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   On a newer Linux Desktop:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ## A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   On a newer Linux Desktop:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ## A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.
   
   ## cargo criterion --bench sort_limit_query_sql
   
   Without this PR:
   ```
   sort_and_limit_by_int   time:   [869.27 us 870.72 us 872.19 us]                                  
   
   sort_and_limit_by_float time:   [845.95 us 847.36 us 848.77 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [875.07 us 876.80 us 878.59 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [878.78 us 880.40 us 882.06 us]
   
   ```
   
   With this PR:
   
   ```
   sort_and_limit_by_int   time:   [856.71 us 858.58 us 860.41 us]                                  
   
   sort_and_limit_by_float time:   [836.55 us 838.40 us 840.29 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [864.83 us 866.33 us 867.83 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [866.89 us 868.25 us 869.62 us]
   
   ```
   
   Similar performance as well.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   On a newer Linux Desktop:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen edited a comment on pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen edited a comment on pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#issuecomment-1085966913


   Hardware Settings:
   
   ```
   H/W path                  Device          Class          Description
   ====================================================================
                                             system         MS-7D53 (To be filled by O.E.M.)
   /0                                        bus            MPG X570S EDGE MAX WIFI (MS-7D53)
   /0/0                                      memory         64KiB BIOS
   /0/11                                     memory         32GiB System Memory
   /0/11/0                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/1                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/11/2                                   memory         3600 MHz (0.3 ns) [empty]
   /0/11/3                                   memory         16GiB DIMM DDR4 Synchronous Unbuffered (Unregistered) 3600 MHz (0.3 
   /0/14                                     memory         1MiB L1 cache
   /0/15                                     memory         8MiB L2 cache
   /0/16                                     memory         64MiB L3 cache
   /0/17                                     processor      AMD Ryzen 9 5950X 16-Core Processor
   
   ```
   
   ## A modified version of TPC-H q1:
   
   ```sql
   select
       l_returnflag,
       l_linestatus,
       l_quantity,
       l_extendedprice,
       l_discount,
       l_tax
   from
       lineitem
   order by
       l_extendedprice,
       l_discount;
   ```
   
   
   ```
   cargo run --release --features "mimalloc" --bin tpch -- benchmark datafusion --iterations 3 --path ../../tpch-parquet/ --format parquet --query 1 --batch-size 4096
   ```
   
   Without this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2851.7 ms and returned 6001214 rows
   Query 1 iteration 1 took 2817.7 ms and returned 6001214 rows
   Query 1 iteration 2 took 2735.9 ms and returned 6001214 rows
   Query 1 avg time: 2801.75 ms
   
   ```
   
   With this PR:
   
   ```
   Running benchmarks with the following options: DataFusionBenchmarkOpt { query: 1, debug: false, iterations: 3, partitions: 2, batch_size: 4096, path: "../../tpch-parquet/", file_format: "parquet", mem_table: false, output_path: None }
   Query 1 iteration 0 took 2843.6 ms and returned 6001214 rows
   Query 1 iteration 1 took 2824.4 ms and returned 6001214 rows
   Query 1 iteration 2 took 2753.0 ms and returned 6001214 rows
   Query 1 avg time: 2807.00 ms
   
   ```
   
   Performances are similar.
   
   ## cargo criterion --bench sort_limit_query_sql
   
   Without this PR:
   ```
   sort_and_limit_by_int   time:   [869.27 us 870.72 us 872.19 us]                                  
   
   sort_and_limit_by_float time:   [845.95 us 847.36 us 848.77 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [875.07 us 876.80 us 878.59 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [878.78 us 880.40 us 882.06 us]
   
   ```
   
   With this PR:
   
   ```
   sort_and_limit_by_int   time:   [856.71 us 858.58 us 860.41 us]                                  
   
   sort_and_limit_by_float time:   [836.55 us 838.40 us 840.29 us]                                    
   
   sort_and_limit_lex_by_int                                                                            
                           time:   [864.83 us 866.33 us 867.83 us]
   
   sort_and_limit_lex_by_string                                                                            
                           time:   [866.89 us 868.25 us 869.62 us]
   
   ```
   
   Similar performance as well.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow-datafusion] yjshen commented on a change in pull request #2132: WIP: Reduce sort memory usage v1

Posted by GitBox <gi...@apache.org>.
yjshen commented on a change in pull request #2132:
URL: https://github.com/apache/arrow-datafusion/pull/2132#discussion_r841011038



##########
File path: datafusion/core/src/physical_plan/sorts/sort.rs
##########
@@ -271,33 +291,212 @@ fn in_mem_partial_sort(
     buffered_batches: &mut Vec<RecordBatch>,
     schema: SchemaRef,
     expressions: &[PhysicalSortExpr],
+    batch_size: usize,
     tracking_metrics: MemTrackingMetrics,
 ) -> Result<SendableRecordBatchStream> {
     assert_ne!(buffered_batches.len(), 0);
+    if buffered_batches.len() == 1 {
+        let result = buffered_batches.pop();
+        Ok(Box::pin(SizedRecordBatchStream::new(
+            schema,
+            vec![Arc::new(result.unwrap())],
+            tracking_metrics,
+        )))
+    } else {
+        let batches = buffered_batches.drain(..).collect::<Vec<_>>();
+        let sorted_iter = {
+            // NB timer records time taken on drop, so there are no
+            // calls to `timer.done()` below.
+            let _timer = tracking_metrics.elapsed_compute().timer();
+            get_sorted_iter(&batches, expressions, batch_size)?
+        };
+        Ok(Box::pin(SortedSizedRecordBatchStream::new(
+            schema,
+            batches,
+            sorted_iter,
+            tracking_metrics,
+        )))
+    }
+}
 
-    let result = {
-        // NB timer records time taken on drop, so there are no
-        // calls to `timer.done()` below.
-        let _timer = tracking_metrics.elapsed_compute().timer();
+fn get_sorted_iter(

Review comment:
       The main changes:
   
   1. concat all sort-columns (instead of all columns)
   2. sort to get the index array (same as the original sort)
   3. use `CombinedIndex` to avoid huge batch construction  (to access records scattered in different batches)
   4. construct a small batch at a time.




-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org