ozankabak commented on code in PR #6742:
URL: https://github.com/apache/arrow-datafusion/pull/6742#discussion_r1240933395


##########
datafusion/core/src/physical_plan/repartition/sort_preserving_repartition.rs:
##########
@@ -0,0 +1,357 @@
+// 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.
+
+//! This file implements an order-preserving repartitioning operator
+//! mapping N input partitions to M output partitions.
+
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+use std::{any::Any, vec};
+
+use super::metrics::{ExecutionPlanMetricsSet, MetricsSet};
+use super::SendableRecordBatchStream;
+
+use crate::physical_plan::common::{
+    transpose, AbortOnDropMany, AbortOnDropSingle, SharedMemoryReservation,
+};
+use crate::physical_plan::metrics::BaselineMetrics;
+use crate::physical_plan::repartition::distributor_channels::{
+    partition_aware_channels, DistributionReceiver, DistributionSender,
+};
+use crate::physical_plan::repartition::{
+    pull_from_input, wait_for_task, MaybeBatch, RepartitionMetrics,
+};
+use crate::physical_plan::sorts::streaming_merge;
+use crate::physical_plan::{
+    DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning,
+    RecordBatchStream, Statistics,
+};
+
+use arrow::datatypes::SchemaRef;
+use arrow::record_batch::RecordBatch;
+use datafusion_common::Result;
+use datafusion_execution::memory_pool::MemoryConsumer;
+use datafusion_execution::TaskContext;
+use datafusion_physical_expr::PhysicalSortExpr;
+
+use futures::{FutureExt, Stream};
+use hashbrown::HashMap;
+use log::trace;
+use parking_lot::Mutex;
+use tokio::task::JoinHandle;
+
+type InputPartitionsToCurrentPartitionSender = 
Vec<DistributionSender<MaybeBatch>>;
+type InputPartitionsToCurrentPartitionReceiver = 
Vec<DistributionReceiver<MaybeBatch>>;
+
+/// Inner state of [`SortPreservingRepartitionExec`].
+#[derive(Debug)]
+struct SortPreservingRepartitionExecState {

Review Comment:
   @crepererum, what do you think about @alamb's suggestion? Can we apply DRY 
here? Assuming we can, could that have any negative performance impact on the 
unordered case, or not?
   
   IIRC you worked on this operator before so maybe you'd have useful insights 
here.



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

To unsubscribe, e-mail: [email protected]

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

Reply via email to