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 2020/12/24 05:02:29 UTC

[GitHub] [arrow] jorgecarleitao commented on a change in pull request #8982: ARROW-10582: [Rust] [DataFusion] Implement "repartition" operator

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



##########
File path: rust/datafusion/src/dataframe.rs
##########
@@ -172,6 +174,24 @@ pub trait DataFrame {
         right_cols: &[&str],
     ) -> Result<Arc<dyn DataFrame>>;
 
+    /// Repartition a DataFrame based on a logical partitioning scheme.
+    ///
+    /// ```
+    /// # use datafusion::prelude::*;
+    /// # use datafusion::error::Result;
+    /// # fn main() -> Result<()> {
+    /// let mut ctx = ExecutionContext::new();
+    /// let df = ctx.read_csv("tests/example.csv", CsvReadOptions::new())?;
+    /// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
+    /// let df2 = df.repartition(Partitioning::Hash(vec![col("a")], 4))?;

Review comment:
       I would not place it in an example since we do not support it yet.

##########
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:
       nit: this introduces a new naming, `partitioning_scheme`.
   
   We have:
   
   * `partition`
   * `partitioning`
   * `partitioning_scheme`
   * `repartition`
   * `part`
   
   I do not know the common notation, but we could try to reduce the number of different names we use.
   
   In my (little) understanding:
   * data is partitioned according to a `partition`
   * partitioned data is divided in `part`s 
   * we can `repartition` it according to a new `partition`.
   
   In this understanding, I would replace `partitioning` and `partitioning_scheme` by `partition`.
   
   Even if this understanding is not correct, maybe we could reduce the number of different names?

##########
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:
       Document `usize`? (Number of parts?)

##########
File path: rust/datafusion/src/physical_plan/repartition.rs
##########
@@ -0,0 +1,336 @@
+// 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;
+
+/// 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,
+    /// Receivers for output batches
+    rx: Arc<Mutex<Vec<Receiver<Option<ArrowResult<RecordBatch>>>>>>,
+    /// Senders for output batches
+    tx: Arc<Mutex<Vec<Sender<Option<ArrowResult<RecordBatch>>>>>>,
+}
+
+#[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 tx = self.tx.lock().await;
+        let mut rx = self.rx.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 tx.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.
+                let (sender, receiver) = unbounded::<Option<ArrowResult<RecordBatch>>>();
+                tx.push(sender);
+                rx.push(receiver);
+            }
+            // launch one async task per *input* partition
+            for i in 0..num_input_partitions {
+                let input = self.input.clone();
+                let mut tx = tx.clone();
+                let partitioning = self.partitioning.clone();
+                let _: JoinHandle<Result<()>> = tokio::spawn(async move {
+                    let mut stream = input.execute(i).await?;
+                    let mut counter = 0;
+                    while let Some(result) = stream.next().await {
+                        match partitioning {
+                            Partitioning::RoundRobinBatch(_) => {

Review comment:
       I have an old hash repartitioning code in a branch around from a previous try. Quite old by now, but I can definitely put it together for this (like I did for the join). I think we now actually have the framework in place to use it.




----------------------------------------------------------------
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:
users@infra.apache.org