alamb commented on a change in pull request #8982:
URL: https://github.com/apache/arrow/pull/8982#discussion_r548508923



##########
File path: rust/datafusion/src/physical_plan/repartition.rs
##########
@@ -0,0 +1,333 @@
+// 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.
+
+//! The repartition operator maps N input partitions to M output partitions 
based on a
+//! partitioning scheme.
+
+use std::any::Any;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+use crate::error::{DataFusionError, Result};
+use crate::physical_plan::{ExecutionPlan, Partitioning};
+use arrow::datatypes::SchemaRef;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use super::{RecordBatchStream, SendableRecordBatchStream};
+use async_trait::async_trait;
+
+use crossbeam::channel::{unbounded, Receiver, Sender};
+use futures::stream::Stream;
+use futures::StreamExt;
+use tokio::sync::Mutex;
+use tokio::task::JoinHandle;
+
+type MaybeBatch = Option<ArrowResult<RecordBatch>>;
+
+/// partition. No guarantees are made about the order of the resulting 
partition.
+#[derive(Debug)]
+pub struct RepartitionExec {
+    /// Input execution plan
+    input: Arc<dyn ExecutionPlan>,
+    /// Partitioning scheme to use
+    partitioning: Partitioning,
+    /// Channels for sending batches from input partitions to output partitions
+    channels: Arc<Mutex<Vec<(Sender<MaybeBatch>, Receiver<MaybeBatch>)>>>,

Review comment:
       ```suggestion
       /// Channels for sending batches from input partitions to output 
partitions
       /// there is one entry in this Vec for each output partition
       channels: Arc<Mutex<Vec<(Sender<MaybeBatch>, Receiver<MaybeBatch>)>>>,
   ```

##########
File path: rust/datafusion/src/execution/dataframe_impl.rs
##########
@@ -111,6 +112,16 @@ impl DataFrame for DataFrameImpl {
         Ok(Arc::new(DataFrameImpl::new(self.ctx_state.clone(), &plan)))
     }
 
+    fn repartition(
+        &self,
+        partitioning_scheme: Partitioning,

Review comment:
       I agree keeping the number of different names low is important 
   
   I suggest using
   * `partition` to refer to an actual portion of the data (in a bunch of 
`RecordBatch`es)
   * `partitioning` to refer to the "schema" of how the data is divided into 
`partition`s (the use of the `Partitioning` scheme now)
   
   Thus we would `repartition` the data into a new `partitioning`
   
   

##########
File path: rust/datafusion/src/logical_plan/plan.rs
##########
@@ -198,6 +206,15 @@ impl LogicalPlan {
     }
 }
 
+/// Logical partitioning schemes supported by the repartition operator.
+#[derive(Debug, Clone)]
+pub enum Partitioning {
+    /// Allocate batches using a round-robin algorithm
+    RoundRobinBatch(usize),
+    /// Allocate rows based on a hash of one of more expressions

Review comment:
       Maybe also add a comment here that `Hash` partitioning is not yet 
completely implemented so as to avoid runtime disappointment for someone who 
sees this enum in the code

##########
File path: rust/datafusion/src/physical_plan/repartition.rs
##########
@@ -0,0 +1,333 @@
+// 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.
+
+//! The repartition operator maps N input partitions to M output partitions 
based on a
+//! partitioning scheme.
+
+use std::any::Any;
+use std::pin::Pin;
+use std::sync::Arc;
+use std::task::{Context, Poll};
+
+use crate::error::{DataFusionError, Result};
+use crate::physical_plan::{ExecutionPlan, Partitioning};
+use arrow::datatypes::SchemaRef;
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use super::{RecordBatchStream, SendableRecordBatchStream};
+use async_trait::async_trait;
+
+use crossbeam::channel::{unbounded, Receiver, Sender};
+use futures::stream::Stream;
+use futures::StreamExt;
+use tokio::sync::Mutex;
+use tokio::task::JoinHandle;
+
+type MaybeBatch = Option<ArrowResult<RecordBatch>>;
+
+/// partition. No guarantees are made about the order of the resulting 
partition.
+#[derive(Debug)]
+pub struct RepartitionExec {
+    /// Input execution plan
+    input: Arc<dyn ExecutionPlan>,
+    /// Partitioning scheme to use
+    partitioning: Partitioning,
+    /// Channels for sending batches from input partitions to output partitions
+    channels: Arc<Mutex<Vec<(Sender<MaybeBatch>, Receiver<MaybeBatch>)>>>,
+}
+
+#[async_trait]
+impl ExecutionPlan for RepartitionExec {
+    /// Return a reference to Any that can be used for downcasting
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    /// Get the schema for this execution plan
+    fn schema(&self) -> SchemaRef {
+        self.input.schema()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.input.clone()]
+    }
+
+    fn with_new_children(
+        &self,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        match children.len() {
+            1 => Ok(Arc::new(RepartitionExec::try_new(
+                children[0].clone(),
+                self.partitioning.clone(),
+            )?)),
+            _ => Err(DataFusionError::Internal(
+                "RepartitionExec wrong number of children".to_string(),
+            )),
+        }
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        self.partitioning.clone()
+    }
+
+    async fn execute(&self, partition: usize) -> 
Result<SendableRecordBatchStream> {
+        // lock mutexes
+        let mut channels = self.channels.lock().await;
+
+        let num_input_partitions = 
self.input.output_partitioning().partition_count();
+        let num_output_partitions = self.partitioning.partition_count();
+
+        // if this is the first partition to be invoked then we need to set up 
initial state
+        if channels.is_empty() {
+            // create one channel per *output* partition
+            for _ in 0..num_output_partitions {
+                // Note that this operator uses unbounded channels to avoid 
deadlocks because
+                // the output partitions can be read in any order and this 
could cause input
+                // partitions to be blocked when sending data to output 
receivers that are not
+                // being read yet. This may cause high memory usage if the 
next operator is
+                // reading output partitions in order rather than 
concurrently. One workaround
+                // for this would be to add spill-to-disk capabilities.

Review comment:
       I think the other work around is to ensure that any operator that reads 
from multiple partitions doesn't block waiting for data from partition channel 
if other partitions can produce data. 
   
   With that invariant, the only operators that would need spill to disk would 
be ones that are maintaining the sorted ness (e.g a classic merge)




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

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


Reply via email to