andygrove commented on a change in pull request #8709:
URL: https://github.com/apache/arrow/pull/8709#discussion_r526849347



##########
File path: rust/datafusion/src/physical_plan/hash_join.rs
##########
@@ -0,0 +1,467 @@
+// 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 std::sync::Arc;
+use std::{
+    any::Any,
+    collections::{HashMap, HashSet},
+};
+
+use async_trait::async_trait;
+use futures::{Stream, StreamExt, TryStreamExt};
+
+use arrow::array::{make_array, Array, MutableArrayData};
+use arrow::datatypes::{Schema, SchemaRef};
+use arrow::error::Result as ArrowResult;
+use arrow::record_batch::RecordBatch;
+
+use super::hash_utils::{build_join_schema, check_join_is_valid, JoinHow};
+use super::{expressions::col, hash_aggregate::create_key};
+use crate::error::{DataFusionError, Result};
+
+use super::{
+    group_scalar::GroupByScalar, ExecutionPlan, Partitioning, 
RecordBatchStream,
+    SendableRecordBatchStream,
+};
+
+// An index of (batch, row) uniquely identifying a row in a part.
+type Index = (usize, usize);
+// None represents a null (e.g. in case of a left join, some right indices are 
null)
+type JoinIndex = (usize, usize);
+// A mapping "on" value -> list of row indexes with this key's value
+// E.g. [1, 2] -> [(0, 3), (1, 6), (0, 8)] indicates that (column1, column2) = 
[1, 2] is true
+// for rows 3 and 8 from batch 0 and row 6 from batch 1.
+type JoinHashMap = HashMap<Vec<GroupByScalar>, Vec<Index>>;
+type JoinLeftData = (JoinHashMap, Vec<RecordBatch>);
+
+/// join execution plan executes partitions in parallel and combines them into 
a set of
+/// partitions.
+#[derive(Debug)]
+pub struct HashJoinExec {
+    /// left side
+    left: Arc<dyn ExecutionPlan>,
+    /// right side
+    right: Arc<dyn ExecutionPlan>,
+    /// Set of common columns used to join on
+    on: HashSet<String>,

Review comment:
       After thinking about this some more, I think it might be better to 
address this in the current PR. I would suggest either:
   
   ```rust
   left_keys: Vec<String>
   right_keys: Vec<String>,
   ```
   
   or the more general case:
   
   ```rust
   left_keys: Vec<Arc<dyn PhysicalExpr>>
   right_keys: Vec<Arc<dyn PhysicalExpr>>,
   ```
   
   It should be possible to perform an equi-join using any deterministic 
expression, for example:
   
   ```
   SELECT a.*, b.* FROM a JOIN b on UPPER(a.name) = UPPER(CONCAT(b.first, ' ', 
b.last))
   ```
   




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


Reply via email to