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 2021/04/19 12:27:09 UTC

[GitHub] [arrow-datafusion] alamb commented on a change in pull request #11: [DataFusion] Cross join implementation

alamb commented on a change in pull request #11:
URL: https://github.com/apache/arrow-datafusion/pull/11#discussion_r615789144



##########
File path: datafusion/src/physical_plan/cross_join.rs
##########
@@ -0,0 +1,315 @@
+// 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.
+
+//! Defines the join plan for executing partitions in parallel and then joining the results
+//! into a set of partitions.

Review comment:
       I wonder if this comment is relevant to cross join or if it was accidentally copy/pasted?

##########
File path: datafusion/src/physical_plan/cross_join.rs
##########
@@ -0,0 +1,315 @@
+// 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.
+
+//! Defines the join plan for executing partitions in parallel and then joining the results
+//! into a set of partitions.
+
+use futures::{lock::Mutex, StreamExt};
+use std::{any::Any, sync::Arc, task::Poll};
+
+use crate::physical_plan::memory::MemoryStream;
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use futures::{Stream, TryStreamExt};
+
+use super::{hash_utils::check_join_is_valid, merge::MergeExec};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+use async_trait::async_trait;
+use std::time::Instant;
+
+use super::{ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream};
+use crate::physical_plan::coalesce_batches::concat_batches;
+use log::debug;
+
+/// Data of the left side
+type JoinLeftData = RecordBatch;
+
+/// executes partitions in parallel and combines them into a set of
+/// partitions by combining all values from the left with all values on the right
+#[derive(Debug)]
+pub struct CrossJoinExec {
+    /// left (build) side which gets loaded in memory
+    left: Arc<dyn ExecutionPlan>,
+    /// right (probe) side which are combined with left side
+    right: Arc<dyn ExecutionPlan>,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Build-side data
+    build_side: Arc<Mutex<Option<JoinLeftData>>>,
+}
+
+impl CrossJoinExec {
+    /// Tries to create a new [CrossJoinExec].
+    /// # Error
+    /// This function errors when it is not possible to join the left and right sides on keys `on`.

Review comment:
       I don't think cross joins have `on` clauses, do they?

##########
File path: datafusion/src/physical_plan/cross_join.rs
##########
@@ -0,0 +1,315 @@
+// 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.
+
+//! Defines the join plan for executing partitions in parallel and then joining the results
+//! into a set of partitions.
+
+use futures::{lock::Mutex, StreamExt};
+use std::{any::Any, sync::Arc, task::Poll};
+
+use crate::physical_plan::memory::MemoryStream;
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use futures::{Stream, TryStreamExt};
+
+use super::{hash_utils::check_join_is_valid, merge::MergeExec};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+use async_trait::async_trait;
+use std::time::Instant;
+
+use super::{ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream};
+use crate::physical_plan::coalesce_batches::concat_batches;
+use log::debug;
+
+/// Data of the left side
+type JoinLeftData = RecordBatch;
+
+/// executes partitions in parallel and combines them into a set of
+/// partitions by combining all values from the left with all values on the right
+#[derive(Debug)]
+pub struct CrossJoinExec {
+    /// left (build) side which gets loaded in memory
+    left: Arc<dyn ExecutionPlan>,
+    /// right (probe) side which are combined with left side
+    right: Arc<dyn ExecutionPlan>,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Build-side data
+    build_side: Arc<Mutex<Option<JoinLeftData>>>,
+}
+
+impl CrossJoinExec {
+    /// Tries to create a new [CrossJoinExec].
+    /// # Error
+    /// This function errors when it is not possible to join the left and right sides on keys `on`.
+    pub fn try_new(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+    ) -> Result<Self> {
+        let left_schema = left.schema();
+        let right_schema = right.schema();
+        check_join_is_valid(&left_schema, &right_schema, &[])?;
+
+        let left_schema = left.schema();
+        let left_fields = left_schema.fields().iter();
+        let right_schema = right.schema();
+
+        let right_fields = right_schema.fields().iter();
+
+        // left then right
+        let all_columns = left_fields.chain(right_fields).cloned().collect();

Review comment:
       This logic is duplicated in the `LogicalPlan` as well - I wonder if it would be important to combine (like pass in the `schema` in addition to `left` and `right`)?

##########
File path: datafusion/tests/sql.rs
##########
@@ -1289,14 +1289,46 @@ async fn equijoin_implicit_syntax_reversed() -> Result<()> {
 }
 
 #[tokio::test]
-async fn cartesian_join() -> Result<()> {
-    let ctx = create_join_context("t1_id", "t2_id")?;
+async fn cross_join() -> Result<()> {
+    let mut ctx = create_join_context("t1_id", "t2_id")?;

Review comment:
       Given the logic in the cross join, I wonder if it would make sense to also test the case where multiple record batches are sent in to the build side as well as the probe side. 

##########
File path: datafusion/src/physical_plan/cross_join.rs
##########
@@ -0,0 +1,315 @@
+// 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.
+
+//! Defines the join plan for executing partitions in parallel and then joining the results
+//! into a set of partitions.
+
+use futures::{lock::Mutex, StreamExt};
+use std::{any::Any, sync::Arc, task::Poll};
+
+use crate::physical_plan::memory::MemoryStream;
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use futures::{Stream, TryStreamExt};
+
+use super::{hash_utils::check_join_is_valid, merge::MergeExec};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+use async_trait::async_trait;
+use std::time::Instant;
+
+use super::{ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream};
+use crate::physical_plan::coalesce_batches::concat_batches;
+use log::debug;
+
+/// Data of the left side
+type JoinLeftData = RecordBatch;
+
+/// executes partitions in parallel and combines them into a set of
+/// partitions by combining all values from the left with all values on the right
+#[derive(Debug)]
+pub struct CrossJoinExec {
+    /// left (build) side which gets loaded in memory
+    left: Arc<dyn ExecutionPlan>,
+    /// right (probe) side which are combined with left side
+    right: Arc<dyn ExecutionPlan>,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Build-side data
+    build_side: Arc<Mutex<Option<JoinLeftData>>>,
+}
+
+impl CrossJoinExec {
+    /// Tries to create a new [CrossJoinExec].
+    /// # Error
+    /// This function errors when it is not possible to join the left and right sides on keys `on`.
+    pub fn try_new(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+    ) -> Result<Self> {
+        let left_schema = left.schema();
+        let right_schema = right.schema();
+        check_join_is_valid(&left_schema, &right_schema, &[])?;
+
+        let left_schema = left.schema();
+        let left_fields = left_schema.fields().iter();
+        let right_schema = right.schema();
+
+        let right_fields = right_schema.fields().iter();
+
+        // left then right
+        let all_columns = left_fields.chain(right_fields).cloned().collect();
+
+        let schema = Arc::new(Schema::new(all_columns));
+
+        Ok(CrossJoinExec {
+            left,
+            right,
+            schema,
+            build_side: Arc::new(Mutex::new(None)),
+        })
+    }
+
+    /// left (build) side which gets loaded in memory
+    pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.left
+    }
+
+    /// right side which gets combined with left side
+    pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.right
+    }
+}
+
+#[async_trait]
+impl ExecutionPlan for CrossJoinExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.left.clone(), self.right.clone()]
+    }
+
+    fn with_new_children(
+        &self,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        match children.len() {
+            2 => Ok(Arc::new(CrossJoinExec::try_new(
+                children[0].clone(),
+                children[1].clone(),
+            )?)),
+            _ => Err(DataFusionError::Internal(
+                "CrossJoinExec wrong number of children".to_string(),
+            )),
+        }
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        self.right.output_partitioning()
+    }
+
+    async fn execute(&self, partition: usize) -> Result<SendableRecordBatchStream> {
+        // we only want to compute the build side once
+        let left_data = {
+            let mut build_side = self.build_side.lock().await;
+
+            match build_side.as_ref() {
+                Some(stream) => stream.clone(),
+                None => {
+                    let start = Instant::now();
+
+                    // merge all left parts into a single stream
+                    let merge = MergeExec::new(self.left.clone());
+                    let stream = merge.execute(0).await?;
+
+                    // Load all batches and count the rows
+                    let (batches, num_rows) = stream
+                        .try_fold((Vec::new(), 0usize), |mut acc, batch| async {
+                            acc.1 += batch.num_rows();
+                            acc.0.push(batch);
+                            Ok(acc)
+                        })
+                        .await?;
+                    let merged_batch =
+                        concat_batches(&self.left.schema(), &batches, num_rows)?;
+                    *build_side = Some(merged_batch.clone());
+
+                    debug!(
+                        "Built build-side of cross join containing {} rows in {} ms",
+                        num_rows,
+                        start.elapsed().as_millis()
+                    );
+
+                    merged_batch
+                }
+            }
+        };
+
+        let stream = self.right.execute(partition).await?;
+
+        if left_data.num_rows() == 0 {
+            return Ok(Box::pin(MemoryStream::try_new(
+                vec![],
+                self.schema.clone(),
+                None,
+            )?));
+        }
+
+        Ok(Box::pin(CrossJoinStream {
+            schema: self.schema.clone(),
+            left_data,
+            right: stream,
+            right_batch: Arc::new(std::sync::Mutex::new(None)),
+            left_index: 0,
+            num_input_batches: 0,
+            num_input_rows: 0,
+            num_output_batches: 0,
+            num_output_rows: 0,
+            join_time: 0,
+        }))
+    }
+}
+
+/// A stream that issues [RecordBatch]es as they arrive from the right  of the join.
+struct CrossJoinStream {
+    /// Input schema
+    schema: Arc<Schema>,
+    /// data from the left side
+    left_data: JoinLeftData,
+    /// right
+    right: SendableRecordBatchStream,
+    /// Current value on the left
+    left_index: usize,
+    /// Current batch being processed from the right side
+    right_batch: Arc<std::sync::Mutex<Option<RecordBatch>>>,

Review comment:
       I wonder if we should be using a `tokio::sync::Mutex` or a `futures::Mutex` here instead of `std::sync::Mutex` which could block the tokio executor thread

##########
File path: datafusion/src/physical_plan/cross_join.rs
##########
@@ -0,0 +1,315 @@
+// 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.
+
+//! Defines the join plan for executing partitions in parallel and then joining the results
+//! into a set of partitions.
+
+use futures::{lock::Mutex, StreamExt};
+use std::{any::Any, sync::Arc, task::Poll};
+
+use crate::physical_plan::memory::MemoryStream;
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use futures::{Stream, TryStreamExt};
+
+use super::{hash_utils::check_join_is_valid, merge::MergeExec};
+use crate::{
+    error::{DataFusionError, Result},
+    scalar::ScalarValue,
+};
+use async_trait::async_trait;
+use std::time::Instant;
+
+use super::{ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream};
+use crate::physical_plan::coalesce_batches::concat_batches;
+use log::debug;
+
+/// Data of the left side
+type JoinLeftData = RecordBatch;
+
+/// executes partitions in parallel and combines them into a set of
+/// partitions by combining all values from the left with all values on the right
+#[derive(Debug)]
+pub struct CrossJoinExec {
+    /// left (build) side which gets loaded in memory
+    left: Arc<dyn ExecutionPlan>,
+    /// right (probe) side which are combined with left side
+    right: Arc<dyn ExecutionPlan>,

Review comment:
       My reading of the code below is that it is the *right* side that is loaded into memory and then the *left* is streamed through. 
   
   I think the changes to the planner i this PR are trying to send the smaller input to the left hand input, I think it might make sense to change the implementation to buffer the left side rather than the right side




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