You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "metesynnada (via GitHub)" <gi...@apache.org> on 2023/02/17 11:22:54 UTC

[GitHub] [arrow-datafusion] metesynnada opened a new pull request, #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

metesynnada opened a new pull request, #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322

   # Which issue does this PR close?
   
   Closes #5321.
   
   This is a large pull request, with over 5.4k additions. However, we made sure to include detailed comments and extensive testing, and the changes made break down as follows:
   
   - 308 lines changed in `Cargo.lock`.
   - 2,300 lines of additional testing code.
   - Over 1,000 lines of new or improved comments.
   
   # Rationale for this change
   
   Symmetric Hash Join (SHJ) extends the join capabilities of Datafusion by supporting filter expressions with order guarantees efficiently. This use case arises commonly in time-series contexts; e.g. use cases involving sliding windows. While ordinary hash join remains the preferable option when both sources are finite, the join type can be changed to SHJ using a `PipelineFixer` sub-rule when both sources are unbounded. 
   
   Let’s see how important this feature is: In a typical stream processing library like Apache Flink or Apache Spark, the join operation can be performed using watermarks. Let's examine a query taken from the Apache Spark docstring:
   
   ```sql
   SELECT * FROM left_table, right_table
   WHERE
       left_key = right_key AND
       left_time > right_time - INTERVAL 12 MINUTES AND
       left_time < right_time + INTERVAL 2 HOUR
   ```
   
   > In this query (), say each join side has a time column, named "left_time" and "right_time", and there is a join condition "left_time > right_time - 8 min". While processing, say the watermark on the right input is "12:34". This means that from henceforth, only right inputs rows with "right_time > 12:34" will be processed, and any older rows will be considered as "too late" and therefore dropped. Then, the left side buffer only needs to keep rows where "left_time > right_time - 8 min > 12:34 - 8m > 12:26". That is, the left state watermark is 12:26, and any rows older than that can be dropped from the state. In other words, the operator will discard all states where the timestamp in state value (input rows) < state watermark.
   > 
   
   Actually, this is part of the picture, not the whole. In theory, range-based pruning can be done with any sorted field (not just the watermark field) and with any arbitrary join filter condition that is amenable to this type of data pruning. However, Apache Spark overfits to timestamps and associates the pruning operation with a watermark. Let’s follow a different approach and examine the following query from a more general, first-principles perspective:
   
   ```sql
   SELECT * FROM left_table, right_table
   WHERE
     left_key = right_key AND
     left_sorted > right_sorted - 3 AND
     left_sorted < right_sorted + 10
   ```
   
   If sort orders of the two columns (`left_sorted` and `right_sorted`) are ascending, and the join condition is `left_sorted > right_sorted - 3`, and the latest value on the right input is 1234, then the left side buffer only has to keep rows where `left_sorted > 1231` and any rows coming before this boundary can be dropped from the buffer. Note that this example is in no way specific; similar scenarios can manifest with a variety of orderings and join filter expressions.
   
   ***[Please refer to the blog post for more information.](https://synnada.notion.site/General-purpose-Stream-Joins-via-Pruning-Symmetric-Hash-Joins-2fe26d3127a241e294a0217b1f18603a)***
   
   # What changes are included in this PR?
   
   The main features included in this PR are:
   
   - An initial library for interval arithmetic, which includes basic arithmetic operations (addition and subtraction) and comparison operations (greater than and less than) for integer types, and supports the logical conjunction operator.
   - An API for performing interval calculations, which can be used for other purposes, such as range pruning in Parquet. Within the context of this PR, we use this functionality to calculate filter expression bounds for pruning purposes.
   - A constraint propagation module to construct expression DAGs from `PhysicalExpr` trees and update column bounds efficiently for data pruning purposes.
   - An initial implementation of SHJ, which is limited to the partitioned mode and does not yet have full support for output order information.
   - A new sub rule for `PipelineFixer` to utilize SHJ instead of ordinary Hash Join when joining two (unbounded) streams.
   
   In order to have a PR with a manageable size, some features have been excluded for now, but will be added in the future. These include:
   
   - Improved support for interval arithmetic, such as support for open/closed intervals, multiply/divide operations, additional comparison and logical operators, floating point numbers, and time intervals.
   - Improved constant propagation, including the ability to determine monotonicity properties of complex `PhysicalExpr`s.
   - An improved SHJ algorithm, including support for collect left/right/all modes, intermediate buffers for complex expressions, and an improved output ordering flag.
   
   ## Performance Gains
   
   SHJ not only makes sliding windows pipeline-friendly, it improves execution throughput even in non-streaming cases in many scenarios, thanks to data pruning. Data pruning results in lower memory requirements, and higher cache efficiency, and opens the door to executing joins entirely in memory for large datasets with short sliding window join filters. ***[You can find a detailed performance analysis for various scenarios here.](https://www.notion.so/synnada/General-purpose-Stream-Joins-via-Pruning-Symmetric-Hash-Joins-2fe26d3127a241e294a0217b1f18603a?pvs=4#0da4085680de4424b588cf67b981ce82)***
   
   # Are these changes tested?
   
   Yes, deterministic and fuzzy unit tests are added.
   
   # Are there any user-facing changes?
   
   No backwards-incompatible changes. 
   
   This change simply creates new use cases in streaming applications. Below, we provide several usage patterns we may start to see more often in the wild, given that we have stream join capabilities:
   
   - Marking sources infinite and provide schema:
   
   ```rust
   let fifo_options = CsvReadOptions::new()
       .schema(schema.as_ref())
       .has_header(false) // Optional
       .mark_infinite(true);
   ```
   
   - Specifying ordering for columns where an a-priori order is known:
   
   ```rust
   let file_sort_order = [datafusion_expr::col("col_name")]
       .into_iter()
       .map(|e| {
           let ascending = true;
           let nulls_first = false;
           e.sort(ascending, nulls_first)
       })
       .collect::<Vec<_>>();
   ```
   
   More examples on table registration can be found in the subroutines we employ in the `unbounded_file_with_symmetric_join` test under `datafusion/core/tests/fifo.rs`.
   
   On the query side, one will be able to execute a query like
   
   ```sql
   SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left_table as t1 
   FULL JOIN right_table as t2 
   ON t1.a2 = t2.a2 
   	AND t1.a1 > t2.a1 + 3 
   	AND t1.a1 < t2.a1 + 10
   ```
   
   in a streaming fashion, so we may see some new usage patterns arise at the query level too.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] Dandandan commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "Dandandan (via GitHub)" <gi...@apache.org>.
Dandandan commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1111957319


##########
datafusion/core/src/physical_plan/joins/hash_join.rs:
##########
@@ -742,71 +751,79 @@ fn build_join_indices(
 // "| 13 | 10 | 130 | 12 | 10 | 120 |",
 // "| 9  | 8  | 90  | 8  | 8  | 80  |",
 // "+----+----+-----+----+----+-----+"
-// And the result of left and right indices
-// left indices:  5, 6, 6, 4
-// right indices: 3, 4, 5, 3
-fn build_equal_condition_join_indices(
-    left_data: &JoinLeftData,
-    right: &RecordBatch,
-    left_on: &[Column],
-    right_on: &[Column],
+// And the result of build and probe indices are:
+// Build indices:  5, 6, 6, 4
+// Probe indices: 3, 4, 5, 3
+#[allow(clippy::too_many_arguments)]
+pub fn build_equal_condition_join_indices(
+    build_hashmap: &JoinHashMap,
+    build_input_buffer: &RecordBatch,
+    probe_batch: &RecordBatch,
+    build_on: &[Column],
+    probe_on: &[Column],
     random_state: &RandomState,
     null_equals_null: &bool,
+    hashes_buffer: &mut Vec<u64>,
+    offset: Option<usize>,
 ) -> Result<(UInt64Array, UInt32Array)> {
-    let keys_values = right_on
+    let keys_values = probe_on
         .iter()
-        .map(|c| Ok(c.evaluate(right)?.into_array(right.num_rows())))
+        .map(|c| Ok(c.evaluate(probe_batch)?.into_array(probe_batch.num_rows())))
         .collect::<Result<Vec<_>>>()?;
-    let left_join_values = left_on
+    let build_join_values = build_on
         .iter()
-        .map(|c| Ok(c.evaluate(&left_data.1)?.into_array(left_data.1.num_rows())))
+        .map(|c| {
+            Ok(c.evaluate(build_input_buffer)?
+                .into_array(build_input_buffer.num_rows()))
+        })
         .collect::<Result<Vec<_>>>()?;
-    let hashes_buffer = &mut vec![0; keys_values[0].len()];

Review Comment:
   good to reuse 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119901310


##########
datafusion/physical-expr/src/intervals/interval_aritmetic.rs:
##########
@@ -0,0 +1,533 @@
+// 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.
+
+//! Interval arithmetic library
+
+use std::borrow::Borrow;
+use std::fmt;
+use std::fmt::{Display, Formatter};
+
+use arrow::compute::{cast_with_options, CastOptions};
+use arrow::datatypes::DataType;
+use datafusion_common::{DataFusionError, Result, ScalarValue};
+use datafusion_expr::Operator;
+
+use crate::aggregate::min_max::{max, min};
+
+/// This type represents an interval, which is used to calculate reliable
+/// bounds for expressions. Currently, we only support addition and
+/// subtraction, but more capabilities will be added in the future.
+/// Upper/lower bounds having NULL values indicate an unbounded side. For
+/// example; [10, 20], [10, ∞], [-∞, 100] and [-∞, ∞] are all valid intervals.
+#[derive(Debug, PartialEq, Clone, Eq, Hash)]
+pub struct Interval {
+    pub lower: ScalarValue,
+    pub upper: ScalarValue,
+}
+
+impl Default for Interval {
+    fn default() -> Self {
+        Interval {
+            lower: ScalarValue::Null,
+            upper: ScalarValue::Null,
+        }
+    }
+}
+
+impl Display for Interval {
+    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
+        write!(f, "Interval [{}, {}]", self.lower, self.upper)
+    }
+}
+
+impl Interval {
+    pub(crate) fn cast_to(
+        &self,
+        data_type: &DataType,
+        cast_options: &CastOptions,
+    ) -> Result<Interval> {
+        Ok(Interval {
+            lower: cast_scalar_value(&self.lower, data_type, cast_options)?,
+            upper: cast_scalar_value(&self.upper, data_type, cast_options)?,
+        })
+    }
+
+    pub(crate) fn get_datatype(&self) -> DataType {
+        self.lower.get_datatype()
+    }
+
+    /// Decide if this interval is certainly greater than, possibly greater than,
+    /// or can't be greater than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn gt(&self, other: &Interval) -> Interval {

Review Comment:
   Revamped as well, maybe you want to check it again.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116468631


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {
+    expr: Arc<dyn PhysicalExpr>,
+    data: Option<T>,
+    child_nodes: Vec<ExprTreeNode<T>>,
+}
+
+impl<T> ExprTreeNode<T> {
+    pub fn new(expr: Arc<dyn PhysicalExpr>) -> Self {
+        ExprTreeNode {
+            expr,
+            data: None,
+            child_nodes: vec![],
+        }
+    }
+
+    pub fn expression(&self) -> &Arc<dyn PhysicalExpr> {
+        &self.expr
+    }
+
+    pub fn children(&self) -> Vec<ExprTreeNode<T>> {
+        self.expr
+            .children()
+            .into_iter()
+            .map(ExprTreeNode::new)
+            .collect()
+    }
+}
+
+impl<T: Clone> TreeNodeRewritable for ExprTreeNode<T> {
+    fn map_children<F>(mut self, transform: F) -> Result<Self>
+    where
+        F: FnMut(Self) -> Result<Self>,
+    {
+        self.child_nodes = self
+            .children()
+            .into_iter()
+            .map(transform)
+            .collect::<Result<Vec<_>>>()?;
+        Ok(self)
+    }
+}
+
+/// This function converts the [PhysicalExpr] tree into a DAG by collecting identical
+/// expressions in one node. Caller specifies the node type in this DAG via the
+/// `constructor` argument, which constructs nodes in this DAG from the [ExprTreeNode]
+/// ancillary object.
+pub fn build_dag<T, F>(
+    expr: Arc<dyn PhysicalExpr>,
+    constructor: &F,
+) -> Result<(NodeIndex, StableGraph<T, usize>)>

Review Comment:
   Is there any UT for the `build_dag` ? 
   I try to understand the implementation but there are so many structs involved here and I get lost soon.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1450735167

   Thanks again everyone


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119610748


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   I tried it, but this creates an ownership problem in the closure (since `sub_rule` does not take a reference as its argument). We can make this work if we change the sub-rule return type so that it always returns an `PipelineStatePropagator` (by returning the argument unchanged when nothing is modified), and not an `Option`. Maybe we can explore this in a refactor PR.
   
   BTW, when we were investigating `clone` calls, we came across a similar situation in some use cases involving `transform_up`/`transform_down`. Maybe we can consider all of these things within that refactor PR and discuss in detail. Will keep this in mind, thanks for pointing it out.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115701082


##########
datafusion/physical-expr/src/intervals/mod.rs:
##########
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   > At the moment, the current implementations are somewhat limited in their ability to handle complex expressions, and it is challenging to expand them beyond their current capabilities. However, with the maturation of the interval arithmetics library, we can explore more advanced expression analysis, which holds the potential for significant progress.
   
   I agree about the current approaches being limited. 
   
   It would be great to have the general purpose interval analysis library in this PR underpinning all three uses. 
   
   It would be great to do a proof of concept (not in this PR) of using the interval library in place of the existing pruning or range analysis. When this PR is merged, I'll plan to file a follow on ticket.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1462591453

   Filed https://github.com/apache/arrow-datafusion/issues/5535 to track the work to unify interval analysis


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb merged pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120426868


##########
datafusion/common/src/scalar.rs:
##########
@@ -1019,6 +1019,29 @@ impl ScalarValue {
         Self::List(scalars, Box::new(Field::new("item", child_type, true)))
     }
 
+    // Create a zero value in the given type.
+    pub fn new_zero(datatype: &DataType) -> Result<ScalarValue> {
+        assert!(datatype.is_primitive());
+        Ok(match datatype {
+            DataType::Boolean => ScalarValue::Boolean(Some(false)),
+            DataType::Int8 => ScalarValue::Int8(Some(0)),
+            DataType::Int16 => ScalarValue::Int16(Some(0)),
+            DataType::Int32 => ScalarValue::Int32(Some(0)),
+            DataType::Int64 => ScalarValue::Int64(Some(0)),
+            DataType::UInt8 => ScalarValue::UInt8(Some(0)),
+            DataType::UInt16 => ScalarValue::UInt16(Some(0)),
+            DataType::UInt32 => ScalarValue::UInt32(Some(0)),
+            DataType::UInt64 => ScalarValue::UInt64(Some(0)),
+            DataType::Float32 => ScalarValue::UInt64(Some(0)),

Review Comment:
   Right, that's a typo -- just sent a commit to fix it as well as the new merge conflicts.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119521969


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -49,16 +57,18 @@ impl PipelineFixer {
     }
 }
 type PipelineFixerSubrule =
-    dyn Fn(&PipelineStatePropagator) -> Option<Result<PipelineStatePropagator>>;
+    dyn Fn(PipelineStatePropagator) -> Option<Result<PipelineStatePropagator>>;

Review Comment:
   Sure, will do.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] Dandandan commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "Dandandan (via GitHub)" <gi...@apache.org>.
Dandandan commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1111954855


##########
datafusion/core/src/physical_plan/joins/hash_join.rs:
##########
@@ -84,25 +84,25 @@ use super::{
 };
 use crate::physical_plan::joins::utils::{
     adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices,
-    get_final_indices_from_bit_map, need_produce_result_in_final,
+    get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide,
 };
 use log::debug;
 use std::fmt;
 use std::task::Poll;
 
-// Maps a `u64` hash value based on the left ["on" values] to a list of indices with this key's value.
+// Maps a `u64` hash value based on the build side ["on" values] to a list of indices with this key's value.

Review Comment:
   I like the refactorings / renaming



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116450639


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   Should we move the join selection logic to `JoinSelection` rule ?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114958710


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   Yes, we will enable it for other cases in the follow-on PRs.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119610748


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   I tried it, but this creates an ownership problem in the closure (since `sub_rule` does not take a reference as its argument). We can make this work if we change the sub-rule return type so that it always returns an `PipelineStatePropagator` (by returning the argument unchanged when nothing is modified), and not an `Option`. Maybe we can explore this in a refactor PR. BTW, we came across a similar issue in some use cases involving `tranform_up`/`transform_down`. Maybe we can consider all of those in that PR.
   
   Will keep this in mind, thanks for pointing it out.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119899569


##########
datafusion/physical-expr/src/intervals/test_utils.rs:
##########
@@ -0,0 +1,67 @@
+// 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.
+
+//! Test utilities for the interval arithmetic library
+
+use std::sync::Arc;
+
+use crate::expressions::{BinaryExpr, Literal};
+use crate::PhysicalExpr;
+use datafusion_common::ScalarValue;
+use datafusion_expr::Operator;
+
+#[allow(clippy::too_many_arguments)]
+/// This test function generates a conjunctive statement with two numeric
+/// terms with the following form:
+/// left_col (op_1) a  > right_col (op_2) b AND left_col (op_3) c < right_col (op_4) d
+pub fn gen_conjunctive_numeric_expr(
+    left_col: Arc<dyn PhysicalExpr>,
+    right_col: Arc<dyn PhysicalExpr>,
+    op_1: Operator,
+    op_2: Operator,
+    op_3: Operator,
+    op_4: Operator,
+    a: i32,
+    b: i32,
+    c: i32,
+    d: i32,
+) -> Arc<dyn PhysicalExpr> {
+    let left_and_1 = Arc::new(BinaryExpr::new(
+        left_col.clone(),
+        op_1,
+        Arc::new(Literal::new(ScalarValue::Int32(Some(a)))),

Review Comment:
   Revamped.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120778673


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {

Review Comment:
   Took a note, we will add comments here in the next follow-on PR. Seems like it slipped through the cracks this time.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1436964442

   > @alamb, if you can mention any collaborator who is interested in this side of things, maybe we can make some progress until you can make some time for this.
   
   I think @liukun4515 / @jackwener  / @ygf11 / @mingmwang  have all worked on joins recently, so they may enjoy seeing a symmetric hash join implementation
   
   @Dandandan  has historically worked on the Join operators, so he may have some insight.
   
   In terms of being able to get the bandwidth to review this PR, the blog is a great first step (I look forward to reading it).
   
   Something that may also help would be to break this PR into several smaller PRs if possible (I haven't looked at the code yet so I don't know how easy / hard this would be to do). 
   
   The rationale for more smaller PRs is that they require less contiguous time to review. For me, at least, finding 10 minutes to review a small PR is much easier to find 2 hours to review a larger one. Other reviewers may have a different opinion, of course, but that is mine


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441485114

   Regarding Join pruning, this is usually achieved by Runtime Filters or Dynamic Filters(constructed dynamically during plan time), we also call this Join pushdown, to prune the rows that fall outside the range of possible join key.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116459658


##########
datafusion/physical-expr/src/rewrite.rs:
##########
@@ -113,6 +113,21 @@ pub trait TreeNodeRewritable: Clone {
         Ok(new_node)
     }
 
+    fn mutable_transform_up<F>(self, op: &mut F) -> Result<Self>
+    where
+        F: FnMut(Self) -> Result<Option<Self>>,
+    {
+        let after_op_children =
+            self.map_children(|node| node.mutable_transform_up(op))?;
+
+        let after_op_children_clone = after_op_children.clone();
+        let new_node = match op(after_op_children)? {
+            Some(value) => value,
+            None => after_op_children_clone,
+        };
+        Ok(new_node)
+    }
+

Review Comment:
   I think you can leverage the `transform_using()` (Visitor pattern) to achieve the same effects instead of adding another` mutable_transform()` method. 
   I think the Visitor pattern is more straightforward and easy to understand if we want to collect some infos and modify the plan/structs during the traversal process.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1435184043

   @alamb, if you can mention any collaborator who is interested in this side of things, maybe we can make some progress until you can make some time for this.
   
   BTW, we wrote an accessible (hopefully) companion blog post that gives an overall idea of everything in this PR (link above). I suggest going through that before getting into code, should make reviewing significantly easier.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1450721254

   I resolved the latest conflicts, @alamb this is good to go


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119508328


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -49,16 +57,18 @@ impl PipelineFixer {
     }
 }
 type PipelineFixerSubrule =
-    dyn Fn(&PipelineStatePropagator) -> Option<Result<PipelineStatePropagator>>;
+    dyn Fn(PipelineStatePropagator) -> Option<Result<PipelineStatePropagator>>;

Review Comment:
   Could you please add some comments for the type definition `dyn Fn(PipelineStatePropagator) -> Option<Result<PipelineStatePropagator>>` ?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120399841


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   We have made significant progress in foundational streaming execution support on the Datafusion side over the last few months. For example, you can already mark sources as infinite/unbounded (check out the tests with FIFO files). Every `ExecutionPlan` then propagates this information upwards, so we make adjustments to the plan in an intelligent way depending on whether we are dealing with a stream.
   
   We will write a blog post about this soon, I will share with you when it is done so you can get up to speed on what we've done already. I think it would be also good to have a meeting with everyone interested in this to discuss future plans and designs.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119688691


##########
datafusion/core/src/physical_plan/joins/hash_join_utils.rs:
##########
@@ -0,0 +1,598 @@
+// 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 contains common subroutines for regular and symmetric hash join
+//! related functionality, used both in join calculations and optimization rules.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::usize;
+
+use arrow::datatypes::SchemaRef;
+
+use datafusion_common::DataFusionError;
+use datafusion_physical_expr::expressions::Column;
+use datafusion_physical_expr::intervals::Interval;
+use datafusion_physical_expr::rewrite::TreeNodeRewritable;
+use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr};
+
+use crate::common::Result;
+use crate::physical_plan::joins::utils::{JoinFilter, JoinSide};
+
+fn check_filter_expr_contains_sort_information(
+    expr: &Arc<dyn PhysicalExpr>,
+    reference: &Arc<dyn PhysicalExpr>,
+) -> bool {
+    expr.eq(reference)
+        || expr
+            .children()
+            .iter()
+            .any(|e| check_filter_expr_contains_sort_information(e, reference))
+}
+
+fn collect_columns_recursive(expr: &Arc<dyn PhysicalExpr>, columns: &mut Vec<Column>) {
+    if let Some(column) = expr.as_any().downcast_ref::<Column>() {
+        if !columns.iter().any(|c| c.eq(column)) {
+            columns.push(column.clone())
+        }
+    }
+    expr.children()
+        .iter()
+        .for_each(|e| collect_columns_recursive(e, columns))
+}
+
+fn collect_columns(expr: &Arc<dyn PhysicalExpr>) -> Vec<Column> {
+    let mut columns = vec![];
+    collect_columns_recursive(expr, &mut columns);
+    columns
+}
+
+/// Create a one to one mapping from main columns to filter columns using
+/// filter column indices. A column index looks like:
+/// ```text
+/// ColumnIndex {
+///     index: 0, // field index in main schema
+///     side: JoinSide::Left, // child side
+/// }
+/// ```
+pub fn map_origin_col_to_filter_col(
+    filter: &JoinFilter,
+    schema: &SchemaRef,
+    side: &JoinSide,
+) -> Result<HashMap<Column, Column>> {
+    let filter_schema = filter.schema();
+    let mut col_to_col_map: HashMap<Column, Column> = HashMap::new();
+    for (filter_schema_index, index) in filter.column_indices().iter().enumerate() {
+        if index.side.eq(side) {
+            // Get the main field from column index:
+            let main_field = schema.field(index.index);
+            // Create a column expression:
+            let main_col = Column::new_with_schema(main_field.name(), schema.as_ref())?;
+            // Since the order of by filter.column_indices() is the same with
+            // that of intermediate schema fields, we can get the column directly.
+            let filter_field = filter_schema.field(filter_schema_index);
+            let filter_col = Column::new(filter_field.name(), filter_schema_index);
+            // Insert mapping:
+            col_to_col_map.insert(main_col, filter_col);
+        }
+    }
+    Ok(col_to_col_map)
+}
+
+/// This function analyzes [PhysicalSortExpr] graphs with respect to monotonicity
+/// (sorting) properties. This is necessary since monotonically increasing and/or
+/// decreasing expressions are required when using join filter expressions for
+/// data pruning purposes.
+///
+/// The method works as follows:
+/// 1. Maps the original columns to the filter columns using the `map_origin_col_to_filter_col` function.
+/// 2. Collects all columns in the sort expression using the `PhysicalExprColumnCollector` visitor.
+/// 3. Checks if all columns are included in the `column_mapping_information` map.
+/// 4. If all columns are included, the sort expression is converted into a filter expression using the `transform_up` and `convert_filter_columns` functions.
+/// 5. Searches the converted filter expression in the filter expression using the `check_filter_expr_contains_sort_information`.
+/// 6. If an exact match is encountered, returns the converted filter expression as `Some(Arc<dyn PhysicalExpr>)`.
+/// 7. If all columns are not included or the exact match is not encountered, returns `None`.
+///
+/// Examples:
+/// Consider the filter expression "a + b > c + 10 AND a + b < c + 100".
+/// 1. If the expression "a@ + d@" is sorted, it will not be accepted since the "d@" column is not part of the filter.
+/// 2. If the expression "d@" is sorted, it will not be accepted since the "d@" column is not part of the filter.
+/// 3. If the expression "a@ + b@ + c@" is sorted, all columns are represented in the filter expression. However,
+///    there is no exact match, so this expression does not indicate pruning.
+pub fn convert_sort_expr_with_filter_schema(
+    side: &JoinSide,
+    filter: &JoinFilter,
+    schema: &SchemaRef,
+    sort_expr: &PhysicalSortExpr,
+) -> Result<Option<Arc<dyn PhysicalExpr>>> {

Review Comment:
   Noted.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114958710


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   As I mentioned in the rule-related comment, this limit is temporary. We will enable SHJ for other cases in one of the follow-on PRs πŸš€ 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441830445

   > Regarding Join pruning, in batch SQL systems, this is usually achieved by Runtime Filters or Dynamic Filters(constructed dynamically during plan time), we also call this Join pushdown, to prune the rows that fall outside the range of possible join key.
   
   Another term from the database literature for this concept I believe is "Sideways information passing" -- the canonical paper is https://www.researchgate.net/publication/45597527_Sideways_Information_Passing_for_Push-Style_Query_Processing
   
   I like to think of this as the hash table created for a hash join is passed to the scan, and during the scan rows that would be filtered out of the join (because there are no matching keys) are filtered during the scan. 
   
   While a neat optimization, I am not sure it is relevant for the usecase of this PR


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116448488


##########
datafusion/core/src/execution/context.rs:
##########
@@ -1528,6 +1528,9 @@ impl SessionState {
             // repartitioning and local sorting steps to meet distribution and ordering requirements.
             // Therefore, it should run before EnforceDistribution and EnforceSorting.
             Arc::new(JoinSelection::new()),
+            // Enforce sort before PipelineFixer

Review Comment:
   I think the comment for `PipelineFixer` need to be adjusted.
   
   ```rust
   // If the query is processing infinite inputs, the PipelineFixer rule applies the
   // necessary transformations to make the query runnable (if it is not already runnable).
   // If the query can not be made runnable, the rule emits an error with a diagnostic message.
   // Since the transformations it applies may alter output partitioning properties of operators
   // (e.g. by swapping hash join sides), this rule runs before EnforceDistribution.
   ```



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120399841


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   We have made significant progress in foundational streaming execution support on the Datafusion side over the last few months. For example, you can already mark sources as infinite/unbounded (check out the tests with FIFO files). Every `ExecutionPlan` then propagates this information upwards, so we make adjustments to (or optimize) the plan in an intelligent way depending on whether we are dealing with a stream.
   
   We will write a blog post about this soon, I will share with you when it is done so you can get up to speed on what we've done already. I think it would be also good to have a meeting with everyone interested in this to discuss future plans and designs.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119528713


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   Seems currently we can not move the logic to JoinSelection rule.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119528195


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   Regarding the method `apply_subrules_and_check_finiteness_requirements()`
   
   I would suggest to make the `input` immutable. If there is changes to the plan/struct, return the changed (new)plan/struct.
   
   
   ```rust
   fn apply_subrules_and_check_finiteness_requirements(
       mut input: PipelineStatePropagator,
       physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
   )
   ```
   
   Suggest change to
   
   ```rust
   fn apply_subrules_and_check_finiteness_requirements(
       input: PipelineStatePropagator,
       physical_optimizer_subrules: &[Box<PipelineFixerSubrule>],
   ) -> Result<Option<PipelineStatePropagator>> {
       let after_op =
           physical_optimizer_subrules
               .iter()
               .try_fold(input, |pipeline, sub_rule| {
                   if let Some(value) = sub_rule(&pipeline).transpose()? {
                       Result::<_, DataFusionError>::Ok(value)
                   } else {
                       Result::<_, DataFusionError>::Ok(pipeline)
                   }
               })?;
       check_finiteness_requirements(after_op)
   }
   ```



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119631386


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   Yes, we will be using Ballista. We haven't contributed much to Ballista yet as we are still focusing on fundamental streaming execution primitives, but will increase our focus on Ballista over time.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119901064


##########
datafusion/core/src/physical_plan/memory.rs:
##########
@@ -144,6 +146,25 @@ impl MemoryExec {
             schema,
             projected_schema,
             projection,
+            sort_information: None,
+        })
+    }
+    /// Create a new execution plan for reading in-memory record batches
+    /// The provided `schema` should not have the projection applied. Also, you can specify sort
+    /// information on PhysicalExprs.
+    pub fn try_new_with_sort_information(

Review Comment:
   Revamped.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114945924


##########
datafusion/core/src/execution/context.rs:
##########
@@ -1528,6 +1528,9 @@ impl SessionState {
             // repartitioning and local sorting steps to meet distribution and ordering requirements.
             // Therefore, it should run before EnforceDistribution and EnforceSorting.
             Arc::new(JoinSelection::new()),
+            // Enforce sort before PipelineFixer

Review Comment:
   We haven't focused on optimal rule ordering yet. Right now, we only enable SHJ when joining two unbounded streams, so it has to happen at some point before `PipelineFixer`. As we mature the SHJ implementation, we will enable it even for normal tables when it is appropriate (i.e. yields performance gains). Within that context, we will revisit both rule internal logic and ordering.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1437363201

   > Something that may also help would be to break this PR into several smaller PRs if possible (I haven't looked at the code yet so I don't know how easy / hard this would be to do).
   
   It may be possible to do this if we carve out things like interval arithmetic, constraint propagation etc. into separate PRs. However, that will deprive you of important context (i.e. where/how do we use these things and why do they have the APIs that they have). For this reason, I suggest keeping this in one piece, and following a top-down order to make reviewing easier (and in parts):
   - Starting with the join-related code, it is quite similar to already existing join code and you will see the touch points of lower-level code. This is probably a good break point (e.g. can context switch to something else).
   - Looking at the code dealing with expression DAGs, how they are built & processed, where you will see the touch points of things like interval arithmetic. Again this is potentially a good break point too.
   - Lowest level code such as interval arithmetic
   
   > The rationale for more smaller PRs is that they require less contiguous time to review. For me, at least, finding 10 minutes to review a small PR is much easier to find 2 hours to review a larger one. Other reviewers may have a different opinion, of course, but that is mine
   
   I agree, it comes down to personal work style. Let's play by ear and see if my suggestions above help. If not, we can always try breaking it up. Our goal is to make the review process be efficient/easy, after all πŸ™‚ 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114958220


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {

Review Comment:
   Sure, we will πŸ‘ 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115296675


##########
datafusion/physical-expr/src/intervals/test_utils.rs:
##########
@@ -0,0 +1,67 @@
+// 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.
+
+//! Test utilities for the interval arithmetic library
+
+use std::sync::Arc;
+
+use crate::expressions::{BinaryExpr, Literal};
+use crate::PhysicalExpr;
+use datafusion_common::ScalarValue;
+use datafusion_expr::Operator;
+
+#[allow(clippy::too_many_arguments)]
+/// This test function generates a conjunctive statement with two numeric
+/// terms with the following form:
+/// left_col (op_1) a  > right_col (op_2) b AND left_col (op_3) c < right_col (op_4) d
+pub fn gen_conjunctive_numeric_expr(
+    left_col: Arc<dyn PhysicalExpr>,
+    right_col: Arc<dyn PhysicalExpr>,
+    op_1: Operator,
+    op_2: Operator,
+    op_3: Operator,
+    op_4: Operator,
+    a: i32,
+    b: i32,
+    c: i32,
+    d: i32,
+) -> Arc<dyn PhysicalExpr> {
+    let left_and_1 = Arc::new(BinaryExpr::new(
+        left_col.clone(),
+        op_1,
+        Arc::new(Literal::new(ScalarValue::Int32(Some(a)))),

Review Comment:
   Thanks, I will leverage these methods in other parts of the codes as well.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441990911

   Thank you @alamb! We will resolve the merge conflict, do a final pass addressing the reviews, and will let you know when it is good to go from our perspective.
   
   > The biggest risk of this PR, in my opinion is the accumulated technical debt of a third range analysis library and another join implementation that is not widely used (in my mind SortMergeJoin also falls in this cateogry). However, I am confident given past interactions the synnadai team will help us unify the approaches and so I think we can merge this PR in as is (and file some follow on tickets)
   
   Sounds good! We will close the loop on all these in the upcoming weeks. We already started some of this work (interval library extension and subsequently unification)
   
   > Another thing that worries me, but is not introduced by this PR, is the size of `datafusion-core` (it would be amazing to break out all the physical_plan stuff to a new module) but that is also well beyond the scope of this PR
   
   This sounds good to me too. We will need to do this at some point, physical plan code is getting quite big.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116789040


##########
datafusion/core/src/physical_plan/memory.rs:
##########
@@ -144,6 +146,25 @@ impl MemoryExec {
             schema,
             projected_schema,
             projection,
+            sort_information: None,
+        })
+    }
+    /// Create a new execution plan for reading in-memory record batches
+    /// The provided `schema` should not have the projection applied. Also, you can specify sort
+    /// information on PhysicalExprs.
+    pub fn try_new_with_sort_information(

Review Comment:
   πŸ‘



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119610748


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   I tried it, but this creates an ownership problem in the closure (since `sub_rule` does not take a reference as its argument). We can make this work if we change the sub-rule return type so that it always returns an `PipelineStatePropagator` (by returning the argument unchanged when nothing is modified), and not an `Option`. Maybe we can explore this in a refactor PR.
   
   BTW, when we were investigating `clone` calls, we came across a similar situation in some use cases involving `transform_up`/`transform_down`. Maybe we can consider all of those in that PR and discuss together the details. Will keep this in mind, thanks for pointing it out.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119610748


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   I tried it, but this creates an ownership problem in the closure (since `sub_rule` does not take a reference as its argument). We can make this work if we change the sub-rule return type so that it always returns an `PipelineStatePropagator` (by returning the argument unchanged when nothing is modified), and not an `Option`. Maybe we can explore this in a refactor PR.
   
   BTW, we came across a similar issue in some use cases involving `tranform_up`/`transform_down`. Maybe we can consider all of those in that PR. Will keep this in mind, thanks for pointing it out.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119898906


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {
+    expr: Arc<dyn PhysicalExpr>,
+    data: Option<T>,
+    child_nodes: Vec<ExprTreeNode<T>>,
+}
+
+impl<T> ExprTreeNode<T> {
+    pub fn new(expr: Arc<dyn PhysicalExpr>) -> Self {
+        ExprTreeNode {
+            expr,
+            data: None,
+            child_nodes: vec![],
+        }
+    }
+
+    pub fn expression(&self) -> &Arc<dyn PhysicalExpr> {
+        &self.expr
+    }
+
+    pub fn children(&self) -> Vec<ExprTreeNode<T>> {
+        self.expr
+            .children()
+            .into_iter()
+            .map(ExprTreeNode::new)
+            .collect()
+    }
+}
+
+impl<T: Clone> TreeNodeRewritable for ExprTreeNode<T> {
+    fn map_children<F>(mut self, transform: F) -> Result<Self>
+    where
+        F: FnMut(Self) -> Result<Self>,
+    {
+        self.child_nodes = self
+            .children()
+            .into_iter()
+            .map(transform)
+            .collect::<Result<Vec<_>>>()?;
+        Ok(self)
+    }
+}
+
+/// This function converts the [PhysicalExpr] tree into a DAG by collecting identical
+/// expressions in one node. Caller specifies the node type in this DAG via the
+/// `constructor` argument, which constructs nodes in this DAG from the [ExprTreeNode]
+/// ancillary object.
+pub fn build_dag<T, F>(
+    expr: Arc<dyn PhysicalExpr>,
+    constructor: &F,
+) -> Result<(NodeIndex, StableGraph<T, usize>)>

Review Comment:
   Added.



##########
datafusion/physical-expr/src/rewrite.rs:
##########
@@ -113,6 +113,21 @@ pub trait TreeNodeRewritable: Clone {
         Ok(new_node)
     }
 
+    fn mutable_transform_up<F>(self, op: &mut F) -> Result<Self>
+    where
+        F: FnMut(Self) -> Result<Option<Self>>,
+    {
+        let after_op_children =
+            self.map_children(|node| node.mutable_transform_up(op))?;
+
+        let after_op_children_clone = after_op_children.clone();
+        let new_node = match op(after_op_children)? {
+            Some(value) => value,
+            None => after_op_children_clone,
+        };
+        Ok(new_node)
+    }
+

Review Comment:
   Added, you can check 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119902002


##########
datafusion/physical-expr/src/intervals/interval_aritmetic.rs:
##########
@@ -0,0 +1,533 @@
+// 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.
+
+//! Interval arithmetic library
+
+use std::borrow::Borrow;
+use std::fmt;
+use std::fmt::{Display, Formatter};
+
+use arrow::compute::{cast_with_options, CastOptions};
+use arrow::datatypes::DataType;
+use datafusion_common::{DataFusionError, Result, ScalarValue};
+use datafusion_expr::Operator;
+
+use crate::aggregate::min_max::{max, min};
+
+/// This type represents an interval, which is used to calculate reliable
+/// bounds for expressions. Currently, we only support addition and
+/// subtraction, but more capabilities will be added in the future.
+/// Upper/lower bounds having NULL values indicate an unbounded side. For
+/// example; [10, 20], [10, ∞], [-∞, 100] and [-∞, ∞] are all valid intervals.
+#[derive(Debug, PartialEq, Clone, Eq, Hash)]
+pub struct Interval {
+    pub lower: ScalarValue,
+    pub upper: ScalarValue,
+}
+
+impl Default for Interval {
+    fn default() -> Self {
+        Interval {
+            lower: ScalarValue::Null,
+            upper: ScalarValue::Null,
+        }
+    }
+}
+
+impl Display for Interval {
+    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
+        write!(f, "Interval [{}, {}]", self.lower, self.upper)
+    }
+}
+
+impl Interval {
+    pub(crate) fn cast_to(
+        &self,
+        data_type: &DataType,
+        cast_options: &CastOptions,
+    ) -> Result<Interval> {
+        Ok(Interval {
+            lower: cast_scalar_value(&self.lower, data_type, cast_options)?,
+            upper: cast_scalar_value(&self.upper, data_type, cast_options)?,
+        })
+    }
+
+    pub(crate) fn get_datatype(&self) -> DataType {
+        self.lower.get_datatype()
+    }
+
+    /// Decide if this interval is certainly greater than, possibly greater than,
+    /// or can't be greater than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn gt(&self, other: &Interval) -> Interval {
+        let flags = if !self.upper.is_null()
+            && !other.lower.is_null()
+            && (self.upper <= other.lower)
+        {
+            (false, false)
+        } else if !self.lower.is_null()
+            && !other.upper.is_null()
+            && (self.lower > other.upper)
+        {
+            (true, true)
+        } else {
+            (false, true)
+        };
+        Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        }
+    }
+
+    /// Decide if this interval is certainly less than, possibly less than,
+    /// or can't be less than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn lt(&self, other: &Interval) -> Interval {
+        other.gt(self)
+    }
+
+    /// Decide if this interval is certainly equal to, possibly equal to,
+    /// or can't be equal to `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.    
+    pub(crate) fn equal(&self, other: &Interval) -> Interval {
+        let flags = if !self.lower.is_null()
+            && (self.lower == self.upper)
+            && (other.lower == other.upper)
+            && (self.lower == other.lower)
+        {
+            (true, true)
+        } else if (!self.lower.is_null()
+            && !other.upper.is_null()
+            && (self.lower > other.upper))
+            || (!self.upper.is_null()
+                && !other.lower.is_null()
+                && (self.upper < other.lower))
+        {
+            (false, false)
+        } else {
+            (false, true)
+        };
+        Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        }
+    }
+
+    /// Compute the logical conjunction of this (boolean) interval with the
+    /// given boolean interval.
+    pub(crate) fn and(&self, other: &Interval) -> Result<Interval> {
+        let flags = match (self, other) {
+            (
+                Interval {
+                    lower: ScalarValue::Boolean(Some(lower)),
+                    upper: ScalarValue::Boolean(Some(upper)),
+                },
+                Interval {
+                    lower: ScalarValue::Boolean(Some(other_lower)),
+                    upper: ScalarValue::Boolean(Some(other_upper)),
+                },
+            ) => {
+                if *lower && *other_lower {
+                    (true, true)
+                } else if *upper && *other_upper {
+                    (false, true)
+                } else {
+                    (false, false)
+                }
+            }
+            _ => {
+                return Err(DataFusionError::Internal(
+                    "Incompatible types for logical conjunction".to_string(),
+                ))
+            }
+        };
+        Ok(Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        })
+    }
+
+    /// Compute the intersection of the interval with the given interval.
+    /// If the intersection is empty, return None.
+    pub(crate) fn intersect(&self, other: &Interval) -> Result<Option<Interval>> {
+        let lower = if self.lower.is_null() {
+            other.lower.clone()
+        } else if other.lower.is_null() {
+            self.lower.clone()
+        } else {
+            max(&self.lower, &other.lower)?
+        };
+        let upper = if self.upper.is_null() {
+            other.upper.clone()
+        } else if other.upper.is_null() {
+            self.upper.clone()
+        } else {
+            min(&self.upper, &other.upper)?
+        };
+        Ok(if !lower.is_null() && !upper.is_null() && lower > upper {
+            // This None value signals an empty interval.
+            None
+        } else {
+            Some(Interval { lower, upper })
+        })
+    }
+
+    // Compute the negation of the interval.
+    #[allow(dead_code)]
+    pub(crate) fn arithmetic_negate(&self) -> Result<Interval> {
+        Ok(Interval {
+            lower: self.upper.arithmetic_negate()?,
+            upper: self.lower.arithmetic_negate()?,
+        })
+    }
+
+    /// Add the given interval (`other`) to this interval. Say we have
+    /// intervals [a1, b1] and [a2, b2], then their sum is [a1 + a2, b1 + b2].
+    /// Note that this represents all possible values the sum can take if
+    /// one can choose single values arbitrarily from each of the operands.
+    pub fn add<T: Borrow<Interval>>(&self, other: T) -> Result<Interval> {
+        let rhs = other.borrow();
+        let lower = if self.lower.is_null() || rhs.lower.is_null() {
+            ScalarValue::try_from(self.lower.get_datatype())
+        } else {
+            self.lower.add(&rhs.lower)
+        }?;
+        let upper = if self.upper.is_null() || rhs.upper.is_null() {
+            ScalarValue::try_from(self.upper.get_datatype())
+        } else {
+            self.upper.add(&rhs.upper)
+        }?;
+        Ok(Interval { lower, upper })
+    }
+
+    /// Subtract the given interval (`other`) from this interval. Say we have
+    /// intervals [a1, b1] and [a2, b2], then their sum is [a1 - b2, b1 - a2].
+    /// Note that this represents all possible values the difference can take
+    /// if one can choose single values arbitrarily from each of the operands.
+    pub fn sub<T: Borrow<Interval>>(&self, other: T) -> Result<Interval> {
+        let rhs = other.borrow();
+        let lower = if self.lower.is_null() || rhs.upper.is_null() {
+            ScalarValue::try_from(self.lower.get_datatype())
+        } else {
+            self.lower.sub(&rhs.upper)
+        }?;
+        let upper = if self.upper.is_null() || rhs.lower.is_null() {
+            ScalarValue::try_from(self.upper.get_datatype())
+        } else {
+            self.upper.sub(&rhs.lower)
+        }?;
+        Ok(Interval { lower, upper })
+    }
+}
+
+/// Indicates whether interval arithmetic is supported for the given operator.
+pub fn is_operator_supported(op: &Operator) -> bool {
+    matches!(
+        op,
+        &Operator::Plus
+            | &Operator::Minus
+            | &Operator::And
+            | &Operator::Gt

Review Comment:
   In the short-term, we will add it. We currently implemented an extension on the interval arithmetics library.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1435123487

   This PR looks neat -- can't wait to review it but I am going to be away next week so I may not have the time required to give this a proper review for another week or two.
   
   perhaps some other reviewers can help (or maybe we can break this PR into smaller parts to make it easier to review)


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119528713


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   Seems currently we can not move the `hash_join_convert_symmetric_subrule` logic to the `JoinSelection` rule. But I think the `hash_join_swap_subrule` logic can be moved to 
   the `JoinSelection` rule. Since the original  `JoinSelection` already has logic to swap join sides based on estimated stats/size, it does not take the bounded/unbounded inputs into the consideration. 



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119902549


##########
datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs:
##########
@@ -0,0 +1,2640 @@
+// 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 the symmetric hash join algorithm with range-based
+//! data pruning to join two (potentially infinite) streams.
+//!
+//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate
+//! output ordering) and produces the join output according to the given join
+//! type and other options.
+//!
+//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations
+//! for both its children.
+
+use std::collections::{HashMap, VecDeque};
+use std::fmt;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+use std::task::Poll;
+use std::vec;
+use std::{any::Any, usize};
+
+use ahash::RandomState;
+use arrow::array::{
+    ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray,
+    PrimitiveBuilder,
+};
+use arrow::compute::concat_batches;
+use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use futures::{Stream, StreamExt};
+use hashbrown::{raw::RawTable, HashSet};
+
+use datafusion_common::{utils::bisect, ScalarValue};
+use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval};
+
+use crate::error::{DataFusionError, Result};
+use crate::execution::context::TaskContext;
+use crate::logical_expr::JoinType;
+use crate::physical_plan::{
+    expressions::Column,
+    expressions::PhysicalSortExpr,
+    joins::{
+        hash_join::{build_join_indices, update_hash, JoinHashMap},
+        hash_join_utils::{build_filter_input_order, SortedFilterExpr},
+        utils::{
+            build_batch_from_indices, build_join_schema, check_join_is_valid,
+            combine_join_equivalence_properties, partitioned_join_output_partitioning,
+            ColumnIndex, JoinFilter, JoinOn, JoinSide,
+        },
+    },
+    metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet},
+    DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+/// A symmetric hash join with range conditions is when both streams are hashed on the
+/// join key and the resulting hash tables are used to join the streams.
+/// The join is considered symmetric because the hash table is built on the join keys from both
+/// streams, and the matching of rows is based on the values of the join keys in both streams.
+/// This type of join is efficient in streaming context as it allows for fast lookups in the hash
+/// table, rather than having to scan through one or both of the streams to find matching rows, also it
+/// only considers the elements from the stream that fall within a certain sliding window (w/ range conditions),
+/// making it more efficient and less likely to store stale data. This enables operating on unbounded streaming
+/// data without any memory issues.
+///
+/// For each input stream, create a hash table.
+///   - For each new [RecordBatch] in build side, hash and insert into inputs hash table. Update offsets.
+///   - Test if input is equal to a predefined set of other inputs.
+///   - If so record the visited rows. If the matched row results must be produced (INNER, LEFT), output the [RecordBatch].
+///   - Try to prune other side (probe) with new [RecordBatch].
+///   - If the join type indicates that the unmatched rows results must be produced (LEFT, FULL etc.),
+/// output the [RecordBatch] when a pruning happens or at the end of the data.
+///
+///
+/// ``` text
+///                        +-------------------------+
+///                        |                         |
+///   left stream ---------|  Left OneSideHashJoiner |---+
+///                        |                         |   |
+///                        +-------------------------+   |
+///                                                      |
+///                                                      |--------- Joined output
+///                                                      |
+///                        +-------------------------+   |
+///                        |                         |   |
+///  right stream ---------| Right OneSideHashJoiner |---+
+///                        |                         |
+///                        +-------------------------+
+///
+/// Prune build side when the new RecordBatch comes to the probe side. We utilize interval arithmetic
+/// on JoinFilter's sorted PhysicalExprs to calculate the joinable range.
+///
+///
+///               PROBE SIDE          BUILD SIDE
+///                 BUFFER              BUFFER
+///             +-------------+     +------------+
+///             |             |     |            |    Unjoinable
+///             |             |     |            |    Range
+///             |             |     |            |
+///             |             |  |---------------------------------
+///             |             |  |  |            |
+///             |             |  |  |            |
+///             |             | /   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |    Joinable
+///             |             |/    |            |    Range
+///             |             ||    |            |
+///             |+-----------+||    |            |
+///             || Record    ||     |            |
+///             || Batch     ||     |            |
+///             |+-----------+||    |            |
+///             +-------------+\    +------------+
+///                             |
+///                             \
+///                              |---------------------------------
+///
+///  This happens when range conditions are provided on sorted columns. E.g.
+///
+///        SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_time > right_time - INTERVAL 12 MINUTES AND left_time < right_time + INTERVAL 2 HOUR
+///
+/// or
+///       SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_sorted > right_sorted - 3 AND left_sorted < right_sorted + 10
+///
+/// For general purpose, in the second scenario, when the new data comes to probe side, the conditions can be used to
+/// determine a specific threshold for discarding rows from the inner buffer. For example, if the sort order the
+/// two columns ("left_sorted" and "right_sorted") are ascending (it can be different in another scenarios)
+/// and the join condition is "left_sorted > right_sorted - 3" and the latest value on the right input is 1234, meaning
+/// that the left side buffer must only keep rows where "leftTime > rightTime - 3 > 1234 - 3 > 1231" ,
+/// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending)
+/// than that can be dropped from the inner buffer.
+/// ```
+pub struct SymmetricHashJoinExec {
+    /// Left side stream
+    pub(crate) left: Arc<dyn ExecutionPlan>,
+    /// Right side stream
+    pub(crate) right: Arc<dyn ExecutionPlan>,
+    /// Set of common columns used to join on
+    pub(crate) on: Vec<(Column, Column)>,
+    /// Filters applied when finding matching rows
+    pub(crate) filter: JoinFilter,
+    /// How the join is performed
+    pub(crate) join_type: JoinType,
+    /// Order information of filter expressions
+    sorted_filter_exprs: Vec<SortedFilterExpr>,
+    /// Left required sort
+    left_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Right required sort
+    right_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Expression graph for interval calculations
+    physical_expr_graph: ExprIntervalGraph,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Shares the `RandomState` for the hashing algorithm
+    random_state: RandomState,
+    /// Execution metrics
+    metrics: ExecutionPlanMetricsSet,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    /// If null_equals_null is true, null == null else null != null
+    pub(crate) null_equals_null: bool,
+}
+
+#[derive(Debug)]
+struct SymmetricHashJoinSideMetrics {
+    /// Number of batches consumed by this operator
+    input_batches: metrics::Count,
+    /// Number of rows consumed by this operator
+    input_rows: metrics::Count,
+}
+
+/// Metrics for HashJoinExec
+#[derive(Debug)]
+struct SymmetricHashJoinMetrics {
+    /// Number of left batches/rows consumed by this operator
+    left: SymmetricHashJoinSideMetrics,
+    /// Number of right batches/rows consumed by this operator
+    right: SymmetricHashJoinSideMetrics,
+    /// Number of batches produced by this operator
+    output_batches: metrics::Count,
+    /// Number of rows produced by this operator
+    output_rows: metrics::Count,
+}
+
+impl SymmetricHashJoinMetrics {
+    pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self {
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let left = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let right = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let output_batches =
+            MetricBuilder::new(metrics).counter("output_batches", partition);
+
+        let output_rows = MetricBuilder::new(metrics).output_rows(partition);
+
+        Self {
+            left,
+            right,
+            output_batches,
+            output_rows,
+        }
+    }
+}
+
+impl SymmetricHashJoinExec {
+    /// Tries to create a new [SymmetricHashJoinExec].
+    /// # Error
+    /// This function errors when:
+    /// - It is not possible to join the left and right sides on keys `on`, or
+    /// - It fails to construct [SortedFilterExpr]s, or
+    /// - It fails to create the [ExprIntervalGraph].
+    pub fn try_new(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: &bool,
+    ) -> Result<Self> {
+        let left_schema = left.schema();
+        let right_schema = right.schema();
+
+        // Error out if no "on" contraints are given:
+        if on.is_empty() {
+            return Err(DataFusionError::Plan(
+                "On constraints in SymmetricHashJoinExec should be non-empty".to_string(),
+            ));
+        }
+
+        // Check if the join is valid with the given on constraints:
+        check_join_is_valid(&left_schema, &right_schema, &on)?;
+
+        // Build the join schema from the left and right schemas:
+        let (schema, column_indices) =
+            build_join_schema(&left_schema, &right_schema, join_type);
+
+        // Set a random state for the join:
+        let random_state = RandomState::with_seeds(0, 0, 0, 0);
+
+        // Create an expression DAG for the join filter:
+        let mut physical_expr_graph =
+            ExprIntervalGraph::try_new(filter.expression().clone())?;
+
+        // Interval calculations require each column to exhibit monotonicity
+        // independently. However, a `PhysicalSortExpr` object defines a
+        // lexicographical ordering, so we can only use their first elements.
+        // when deducing column monotonicities.
+        // TODO: Extend the `PhysicalSortExpr` mechanism to express independent
+        //       (i.e. simultaneous) ordering properties of columns.
+        let (left_ordering, right_ordering) = match (
+            left.output_ordering(),
+            right.output_ordering(),
+        ) {
+            (Some([left_ordering, ..]), Some([right_ordering, ..])) => {
+                (left_ordering, right_ordering)
+            }
+            _ => {
+                return Err(DataFusionError::Plan(
+                    "Symmetric hash join requires its children to have an output ordering".to_string(),
+                ));
+            }
+        };
+
+        // Build the sorted filter expression for the left child:
+        let left_filter_expression = build_filter_input_order(
+            JoinSide::Left,
+            &filter,
+            &left.schema(),
+            left_ordering,
+        )?;
+
+        // Build the sorted filter expression for the right child:
+        let right_filter_expression = build_filter_input_order(
+            JoinSide::Right,
+            &filter,
+            &right.schema(),
+            right_ordering,
+        )?;
+
+        // Store the left and right sorted filter expressions in a vector
+        let mut sorted_filter_exprs =
+            vec![left_filter_expression, right_filter_expression];
+
+        // Gather node indices of converted filter expressions in `SortedFilterExpr`
+        // using the filter columns vector:
+        let child_node_indexes = physical_expr_graph.gather_node_indices(
+            &sorted_filter_exprs
+                .iter()
+                .map(|sorted_expr| sorted_expr.filter_expr().clone())
+                .collect::<Vec<_>>(),
+        );
+
+        // Inject calculated node indices into SortedFilterExpr:
+        for (sorted_expr, (_, index)) in sorted_filter_exprs
+            .iter_mut()
+            .zip(child_node_indexes.iter())
+        {
+            sorted_expr.set_node_index(*index);
+        }
+
+        let left_required_sort_exprs = vec![left_ordering.clone()];
+        let right_required_sort_exprs = vec![right_ordering.clone()];
+
+        Ok(SymmetricHashJoinExec {
+            left,
+            right,
+            on,
+            filter,
+            join_type: *join_type,
+            sorted_filter_exprs,
+            left_required_sort_exprs,
+            right_required_sort_exprs,
+            physical_expr_graph,
+            schema: Arc::new(schema),
+            random_state,
+            metrics: ExecutionPlanMetricsSet::new(),
+            column_indices,
+            null_equals_null: *null_equals_null,
+        })
+    }
+
+    /// left stream
+    pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.left
+    }
+
+    /// right stream
+    pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.right
+    }
+
+    /// Set of common columns used to join on
+    pub fn on(&self) -> &[(Column, Column)] {
+        &self.on
+    }
+
+    /// Filters applied before join output
+    pub fn filter(&self) -> &JoinFilter {
+        &self.filter
+    }
+
+    /// How the join is performed
+    pub fn join_type(&self) -> &JoinType {
+        &self.join_type
+    }
+
+    /// Get null_equals_null
+    pub fn null_equals_null(&self) -> &bool {
+        &self.null_equals_null
+    }
+}
+
+impl Debug for SymmetricHashJoinExec {
+    fn fmt(&self, _f: &mut Formatter<'_>) -> fmt::Result {
+        todo!()
+    }
+}
+
+impl ExecutionPlan for SymmetricHashJoinExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
+        vec![
+            Some(&self.left_required_sort_exprs),
+            Some(&self.right_required_sort_exprs),
+        ]
+    }
+
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children.iter().any(|u| *u))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        let (left_expr, right_expr) = self
+            .on
+            .iter()
+            .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
+            .unzip();
+        // TODO: This will change when we extend collected executions.
+        vec![
+            if self.left.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(left_expr)
+            },
+            if self.right.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(right_expr)
+            },
+        ]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        let left_columns_len = self.left.schema().fields.len();
+        partitioned_join_output_partitioning(
+            self.join_type,
+            self.left.output_partitioning(),
+            self.right.output_partitioning(),
+            left_columns_len,
+        )
+    }
+
+    // TODO: Output ordering might be kept for some cases.
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        let left_columns_len = self.left.schema().fields.len();
+        combine_join_equivalence_properties(
+            self.join_type,
+            self.left.equivalence_properties(),
+            self.right.equivalence_properties(),
+            left_columns_len,
+            self.on(),
+            self.schema(),
+        )
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.left.clone(), self.right.clone()]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(SymmetricHashJoinExec::try_new(
+            children[0].clone(),
+            children[1].clone(),
+            self.on.clone(),
+            self.filter.clone(),
+            &self.join_type,
+            &self.null_equals_null,
+        )?))
+    }
+
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                let display_filter = format!(", filter={:?}", self.filter.expression());
+                write!(
+                    f,
+                    "SymmetricHashJoinExec: join_type={:?}, on={:?}{}",
+                    self.join_type, self.on, display_filter
+                )
+            }
+        }
+    }
+
+    fn metrics(&self) -> Option<MetricsSet> {
+        Some(self.metrics.clone_inner())
+    }
+
+    fn statistics(&self) -> Statistics {
+        // TODO stats: it is not possible in general to know the output size of joins
+        Statistics::default()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
+        let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
+        let left_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Left,
+            self.sorted_filter_exprs[0].clone(),
+            on_left,
+            self.left.schema(),
+        );
+        let right_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Right,
+            self.sorted_filter_exprs[1].clone(),
+            on_right,
+            self.right.schema(),
+        );
+        let left_stream = self.left.execute(partition, context.clone())?;
+        let right_stream = self.right.execute(partition, context)?;
+
+        Ok(Box::pin(SymmetricHashJoinStream {
+            left_stream,
+            right_stream,
+            schema: self.schema(),
+            filter: self.filter.clone(),
+            join_type: self.join_type,
+            random_state: self.random_state.clone(),
+            left: left_side_joiner,
+            right: right_side_joiner,
+            column_indices: self.column_indices.clone(),
+            metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics),
+            physical_expr_graph: self.physical_expr_graph.clone(),
+            null_equals_null: self.null_equals_null,
+            final_result: false,
+            probe_side: JoinSide::Left,
+        }))
+    }
+}
+
+/// A stream that issues [RecordBatch]es as they arrive from the right  of the join.
+struct SymmetricHashJoinStream {
+    /// Left stream
+    left_stream: SendableRecordBatchStream,
+    /// right stream
+    right_stream: SendableRecordBatchStream,
+    /// Input schema
+    schema: Arc<Schema>,
+    /// join filter
+    filter: JoinFilter,
+    /// type of the join
+    join_type: JoinType,
+    // left hash joiner
+    left: OneSideHashJoiner,
+    /// right hash joiner
+    right: OneSideHashJoiner,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    // Range pruner.
+    physical_expr_graph: ExprIntervalGraph,
+    /// Random state used for hashing initialization
+    random_state: RandomState,
+    /// If null_equals_null is true, null == null else null != null
+    null_equals_null: bool,
+    /// Metrics
+    metrics: SymmetricHashJoinMetrics,
+    /// Flag indicating whether there is nothing to process anymore
+    final_result: bool,
+    /// The current probe side. We choose build and probe side according to this attribute.
+    probe_side: JoinSide,
+}
+
+impl RecordBatchStream for SymmetricHashJoinStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+impl Stream for SymmetricHashJoinStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Self::Item>> {
+        self.poll_next_impl(cx)
+    }
+}
+
+fn prune_hash_values(
+    prune_length: usize,
+    hashmap: &mut JoinHashMap,
+    row_hash_values: &mut VecDeque<u64>,
+    offset: u64,
+) -> Result<()> {
+    // Create a (hash)-(row number set) map
+    let mut hash_value_map: HashMap<u64, HashSet<u64>> = HashMap::new();
+    for index in 0..prune_length {
+        let hash_value = row_hash_values.pop_front().unwrap();
+        if let Some(set) = hash_value_map.get_mut(&hash_value) {
+            set.insert(offset + index as u64);
+        } else {
+            let mut set = HashSet::new();
+            set.insert(offset + index as u64);
+            hash_value_map.insert(hash_value, set);
+        }
+    }
+    for (hash_value, index_set) in hash_value_map.iter() {
+        if let Some((_, separation_chain)) = hashmap
+            .0
+            .get_mut(*hash_value, |(hash, _)| hash_value == hash)
+        {
+            separation_chain.retain(|n| !index_set.contains(n));
+            if separation_chain.is_empty() {
+                hashmap
+                    .0
+                    .remove_entry(*hash_value, |(hash, _)| hash_value == hash);
+            }
+        }
+    }
+    Ok(())
+}
+
+/// Calculate the filter expression intervals.
+///
+/// This function updates the `interval` field of each `SortedFilterExpr` based
+/// on the first or the last value of the expression in `build_input_buffer`
+/// and `probe_batch`.
+///
+/// # Arguments
+///
+/// * `build_input_buffer` - The [RecordBatch] on the build side of the join.
+/// * `build_sorted_filter_expr` - Build side [SortedFilterExpr] to update.
+/// * `probe_batch` - The `RecordBatch` on the probe side of the join.
+/// * `probe_sorted_filter_expr` - Probe side `SortedFilterExpr` to update.
+///
+/// ### Note
+/// ```text
+///
+/// Interval arithmetic is used to calculate viable join ranges for build-side
+/// pruning. This is done by first creating an interval for join filter values in
+/// the build side of the join, which spans [-∞, FV] or [FV, ∞] depending on the
+/// ordering (descending/ascending) of the filter expression. Here, FV denotes the
+/// first value on the build side. This range is then compared with the probe side
+/// interval, which either spans [-∞, LV] or [LV, ∞] depending on the ordering
+/// (ascending/descending) of the probe side. Here, LV denotes the last value on
+/// the probe side.
+///
+/// As a concrete example, consider the following query:
+///
+///   SELECT * FROM left_table, right_table
+///   WHERE
+///     left_key = right_key AND
+///     a > b - 3 AND
+///     a < b + 10
+///
+/// where columns "a" and "b" come from tables "left_table" and "right_table",
+/// respectively. When a new `RecordBatch` arrives at the right side, the
+/// condition a > b - 3 will possibly indicate a prunable range for the left
+/// side. Conversely, when a new `RecordBatch` arrives at the left side, the
+/// condition a < b + 10 will possibly indicate prunability for the right side.
+/// Let’s inspect what happens when a new RecordBatch` arrives at the right
+/// side (i.e. when the left side is the build side):
+///
+///         Build      Probe
+///       +-------+  +-------+
+///       | a | z |  | b | y |
+///       |+--|--+|  |+--|--+|
+///       | 1 | 2 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 3 | 1 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 5 | 7 |  | 6 | 1 |
+///       |+--|--+|  |+--|--+|
+///       | 7 | 1 |  | 6 | 3 |
+///       +-------+  +-------+
+///
+/// In this case, the interval representing viable (i.e. joinable) values for
+/// column "a" is [1, ∞], and the interval representing possible future values
+/// for column "b" is [6, ∞]. With these intervals at hand, we next calculate
+/// intervals for the whole filter expression and propagate join constraint by
+/// traversing the expression graph.
+/// ```
+fn calculate_filter_expr_intervals(
+    build_input_buffer: &RecordBatch,
+    build_sorted_filter_expr: &mut SortedFilterExpr,
+    probe_batch: &RecordBatch,
+    probe_sorted_filter_expr: &mut SortedFilterExpr,
+) -> Result<()> {
+    // If either build or probe side has no data, return early:
+    if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
+        return Ok(());
+    }
+    // Evaluate build side filter expression and convert the result to an array
+    let build_array = build_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&build_input_buffer.slice(0, 1))?
+        .into_array(1);
+    // Evaluate probe side filter expression and convert the result to an array
+    let probe_array = probe_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))?
+        .into_array(1);
+
+    // Update intervals for both build and probe side filter expressions
+    for (array, sorted_expr) in vec![
+        (build_array, build_sorted_filter_expr),
+        (probe_array, probe_sorted_filter_expr),
+    ] {
+        // Convert the array to a ScalarValue:
+        let value = ScalarValue::try_from_array(&array, 0)?;
+        // Create a ScalarValue representing positive or negative infinity for the same data type:
+        let infinite = ScalarValue::try_from(value.get_datatype())?;
+        // Update the interval with lower and upper bounds based on the sort option
+        sorted_expr.set_interval(
+            if sorted_expr.origin_sorted_expr().options.descending {
+                Interval {
+                    lower: infinite,
+                    upper: value,
+                }
+            } else {
+                Interval {
+                    lower: value,
+                    upper: infinite,
+                }
+            },
+        );
+    }
+    Ok(())
+}
+
+/// Determine the pruning length for `buffer`.
+///
+/// This function evaluates the build side filter expression, converts the
+/// result into an array and determines the pruning length by performing a
+/// binary search on the array.
+///
+/// # Arguments
+///
+/// * `buffer`: The record batch to be pruned.
+/// * `build_side_filter_expr`: The filter expression on the build side used
+/// to determine the pruning length.
+///
+/// # Returns
+///
+/// A [Result] object that contains the pruning length. The function will return
+/// an error if there is an issue evaluating the build side filter expression.
+fn determine_prune_length(
+    buffer: &RecordBatch,
+    build_side_filter_expr: &SortedFilterExpr,
+) -> Result<usize> {
+    let origin_sorted_expr = build_side_filter_expr.origin_sorted_expr();
+    let interval = build_side_filter_expr.interval();
+    // Evaluate the build side filter expression and convert it into an array
+    let batch_arr = origin_sorted_expr
+        .expr
+        .evaluate(buffer)?
+        .into_array(buffer.num_rows());
+
+    // Get the lower or upper interval based on the sort direction
+    let target = if origin_sorted_expr.options.descending {
+        interval.upper.clone()
+    } else {
+        interval.lower.clone()
+    };
+
+    // Perform binary search on the array to determine the length of the record batch to be pruned
+    bisect::<true>(&[batch_arr], &[target], &[origin_sorted_expr.options])
+}
+
+/// This method determines if the result of the join should be produced in the final step or not.
+///
+/// # Arguments
+///
+/// * `build_side` - Enum indicating the side of the join used as the build side.
+/// * `join_type` - Enum indicating the type of join to be performed.
+///
+/// # Returns
+///
+/// A boolean indicating whether the result of the join should be produced in the final step or not.
+/// The result will be true if the build side is JoinSide::Left and the join type is one of
+/// JoinType::Left, JoinType::LeftAnti, JoinType::Full or JoinType::LeftSemi.
+/// If the build side is JoinSide::Right, the result will be true if the join type
+/// is one of JoinType::Right, JoinType::RightAnti, JoinType::Full, or JoinType::RightSemi.
+fn need_to_produce_result_in_final(build_side: JoinSide, join_type: JoinType) -> bool {
+    if build_side == JoinSide::Left {
+        matches!(
+            join_type,
+            JoinType::Left | JoinType::LeftAnti | JoinType::Full | JoinType::LeftSemi
+        )
+    } else {
+        matches!(
+            join_type,
+            JoinType::Right | JoinType::RightAnti | JoinType::Full | JoinType::RightSemi
+        )
+    }
+}
+
+/// Get the anti join indices from the visited hash set.
+///
+/// This method returns the indices from the original input that were not present in the visited hash set.
+///
+/// # Arguments
+///
+/// * `prune_length` - The length of the pruned record batch.
+/// * `deleted_offset` - The offset to the indices.
+/// * `visited_rows` - The hash set of visited indices.
+///
+/// # Returns
+///
+/// A `PrimitiveArray` of the anti join indices.
+fn get_anti_indices<T: ArrowPrimitiveType>(
+    prune_length: usize,
+    deleted_offset: usize,
+    visited_rows: &HashSet<usize>,
+) -> PrimitiveArray<T>
+where
+    NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
+{
+    let mut bitmap = BooleanBufferBuilder::new(prune_length);
+    bitmap.append_n(prune_length, false);
+    // mark the indices as true if they are present in the visited hash set
+    for v in 0..prune_length {
+        let row = v + deleted_offset;
+        bitmap.set_bit(v, visited_rows.contains(&row));
+    }
+    // get the anti index
+    (0..prune_length)
+        .filter_map(|idx| (!bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
+        .collect()
+}
+
+/// This method creates a boolean buffer from the visited rows hash set
+/// and the indices of the pruned record batch slice.
+///
+/// It gets the indices from the original input that were present in the visited hash set.
+///
+/// # Arguments
+///
+/// * `prune_length` - The length of the pruned record batch.
+/// * `deleted_offset` - The offset to the indices.
+/// * `visited_rows` - The hash set of visited indices.
+///
+/// # Returns
+///
+/// A [PrimitiveArray] of the specified type T, containing the semi indices.
+fn get_semi_indices<T: ArrowPrimitiveType>(
+    prune_length: usize,
+    deleted_offset: usize,
+    visited_rows: &HashSet<usize>,
+) -> PrimitiveArray<T>
+where
+    NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
+{
+    let mut bitmap = BooleanBufferBuilder::new(prune_length);
+    bitmap.append_n(prune_length, false);
+    // mark the indices as true if they are present in the visited hash set
+    (0..prune_length).for_each(|v| {
+        let row = &(v + deleted_offset);
+        bitmap.set_bit(v, visited_rows.contains(row));
+    });
+    // get the semi index
+    (0..prune_length)
+        .filter_map(|idx| (bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
+        .collect::<PrimitiveArray<T>>()
+}
+/// Records the visited indices from the input `PrimitiveArray` of type `T` into the given hash set `visited`.
+/// This function will insert the indices (offset by `offset`) into the `visited` hash set.
+///
+/// # Arguments
+///
+/// * `visited` - A hash set to store the visited indices.
+/// * `offset` - An offset to the indices in the `PrimitiveArray`.
+/// * `indices` - The input `PrimitiveArray` of type `T` which stores the indices to be recorded.
+///
+fn record_visited_indices<T: ArrowPrimitiveType>(
+    visited: &mut HashSet<usize>,
+    offset: usize,
+    indices: &PrimitiveArray<T>,
+) {
+    for i in indices.values() {
+        visited.insert(i.as_usize() + offset);
+    }
+}
+
+/// Calculate indices by join type.
+///
+/// This method returns a tuple of two arrays: build and probe indices.
+/// The length of both arrays will be the same.
+///
+/// # Arguments
+///
+/// * `build_side`: Join side which defines the build side.
+/// * `prune_length`: Length of the prune data.
+/// * `visited_rows`: Hash set of visited rows of the build side.
+/// * `deleted_offset`: Deleted offset of the build side.
+/// * `join_type`: The type of join to be performed.
+///
+/// # Returns
+///
+/// A tuple of two arrays of primitive types representing the build and probe indices.
+///
+fn calculate_indices_by_join_type<L: ArrowPrimitiveType, R: ArrowPrimitiveType>(
+    build_side: JoinSide,
+    prune_length: usize,
+    visited_rows: &HashSet<usize>,
+    deleted_offset: usize,
+    join_type: JoinType,
+) -> Result<(PrimitiveArray<L>, PrimitiveArray<R>)>
+where
+    NativeAdapter<L>: From<<L as ArrowPrimitiveType>::Native>,
+{
+    // Store the result in a tuple
+    let result = match (build_side, join_type) {
+        // In the case of `Left` or `Right` join, or `Full` join, get the anti indices
+        (JoinSide::Left, JoinType::Left | JoinType::LeftAnti)
+        | (JoinSide::Right, JoinType::Right | JoinType::RightAnti)
+        | (_, JoinType::Full) => {
+            let build_unmatched_indices =
+                get_anti_indices(prune_length, deleted_offset, visited_rows);
+            let mut builder =
+                PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
+            builder.append_nulls(build_unmatched_indices.len());
+            let probe_indices = builder.finish();
+            (build_unmatched_indices, probe_indices)
+        }
+        // In the case of `LeftSemi` or `RightSemi` join, get the semi indices
+        (JoinSide::Left, JoinType::LeftSemi) | (JoinSide::Right, JoinType::RightSemi) => {
+            let build_unmatched_indices =
+                get_semi_indices(prune_length, deleted_offset, visited_rows);
+            let mut builder =
+                PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
+            builder.append_nulls(build_unmatched_indices.len());
+            let probe_indices = builder.finish();
+            (build_unmatched_indices, probe_indices)
+        }
+        // The case of other join types is not considered
+        _ => unreachable!(),
+    };
+    Ok(result)
+}
+
+struct OneSideHashJoiner {
+    /// Build side
+    build_side: JoinSide,
+    /// Build side filter sort information
+    sorted_filter_expr: SortedFilterExpr,
+    /// Input record batch buffer
+    input_buffer: RecordBatch,
+    /// Columns from the side
+    on: Vec<Column>,
+    /// Hashmap
+    hashmap: JoinHashMap,
+    /// To optimize hash deleting in case of pruning, we hold them in memory
+    row_hash_values: VecDeque<u64>,
+    /// Reuse the hashes buffer
+    hashes_buffer: Vec<u64>,
+    /// Matched rows
+    visited_rows: HashSet<usize>,
+    /// Offset
+    offset: usize,
+    /// Deleted offset
+    deleted_offset: usize,
+    /// Side is exhausted
+    exhausted: bool,
+}
+
+impl OneSideHashJoiner {
+    pub fn new(
+        build_side: JoinSide,
+        sorted_filter_expr: SortedFilterExpr,
+        on: Vec<Column>,
+        schema: SchemaRef,
+    ) -> Self {
+        Self {
+            build_side,
+            input_buffer: RecordBatch::new_empty(schema),
+            on,
+            hashmap: JoinHashMap(RawTable::with_capacity(10_000)),
+            row_hash_values: VecDeque::new(),
+            hashes_buffer: vec![],
+            sorted_filter_expr,
+            visited_rows: HashSet::new(),
+            offset: 0,
+            deleted_offset: 0,
+            exhausted: false,
+        }
+    }
+
+    /// Updates the internal state of the [OneSideHashJoiner] with the incoming batch.
+    ///
+    /// # Arguments
+    ///
+    /// * `batch` - The incoming [RecordBatch] to be merged with the internal input buffer
+    /// * `random_state` - The random state used to hash values
+    ///
+    /// # Returns
+    ///
+    /// Returns a [Result] encapsulating any intermediate errors.
+    fn update_internal_state(
+        &mut self,
+        batch: &RecordBatch,
+        random_state: &RandomState,
+    ) -> Result<()> {
+        // Merge the incoming batch with the existing input buffer:
+        self.input_buffer = concat_batches(&batch.schema(), [&self.input_buffer, batch])?;
+        // Resize the hashes buffer to the number of rows in the incoming batch:
+        self.hashes_buffer.resize(batch.num_rows(), 0);
+        // Update the hashmap with the join key values and hashes of the incoming batch:
+        update_hash(
+            &self.on,
+            batch,
+            &mut self.hashmap,
+            self.offset,
+            random_state,
+            &mut self.hashes_buffer,
+        )?;
+        // Add the hashes buffer to the hash value deque:
+        self.row_hash_values.extend(self.hashes_buffer.iter());
+        Ok(())
+    }
+
+    /// This method performs a join between the build side input buffer and the probe side batch.
+    ///
+    /// # Arguments
+    ///
+    /// * `schema` - A reference to the schema of the output record batch.
+    /// * `join_type` - The type of join to be performed.
+    /// * `on_probe` - An array of columns on which the join will be performed. The columns are from the probe side of the join.
+    /// * `filter` - An optional filter on the join condition.
+    /// * `probe_batch` - The second record batch to be joined.
+    /// * `probe_visited` - A hash set to store the visited indices from the probe batch.
+    /// * `probe_offset` - The offset of the probe side for visited indices calculations.
+    /// * `column_indices` - An array of columns to be selected for the result of the join.
+    /// * `random_state` - The random state for the join.
+    /// * `null_equals_null` - A boolean indicating whether NULL values should be treated as equal when joining.
+    ///
+    /// # Returns
+    ///
+    /// A [Result] containing an optional record batch if the join type is not one of `LeftAnti`, `RightAnti`, `LeftSemi` or `RightSemi`.
+    /// If the join type is one of the above four, the function will return [None].
+    #[allow(clippy::too_many_arguments)]
+    fn join_with_probe_batch(
+        &mut self,
+        schema: &SchemaRef,
+        join_type: JoinType,
+        on_probe: &[Column],
+        filter: &JoinFilter,
+        probe_batch: &RecordBatch,
+        probe_visited: &mut HashSet<usize>,
+        probe_offset: usize,
+        column_indices: &[ColumnIndex],
+        random_state: &RandomState,
+        null_equals_null: &bool,
+    ) -> Result<Option<RecordBatch>> {
+        if self.input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
+            return Ok(Some(RecordBatch::new_empty(schema.clone())));
+        }
+        let (build_indices, probe_indices) = build_join_indices(
+            probe_batch,
+            &self.hashmap,
+            &self.input_buffer,
+            &self.on,
+            on_probe,
+            Some(filter),
+            random_state,
+            null_equals_null,
+            &mut self.hashes_buffer,
+            Some(self.deleted_offset),
+            self.build_side,
+        )?;
+        if need_to_produce_result_in_final(self.build_side, join_type) {
+            record_visited_indices(
+                &mut self.visited_rows,
+                self.deleted_offset,
+                &build_indices,
+            );
+        }
+        if need_to_produce_result_in_final(self.build_side.negate(), join_type) {
+            record_visited_indices(probe_visited, probe_offset, &probe_indices);
+        }
+        if matches!(
+            join_type,
+            JoinType::LeftAnti
+                | JoinType::RightAnti
+                | JoinType::LeftSemi
+                | JoinType::RightSemi
+        ) {
+            Ok(None)
+        } else {
+            build_batch_from_indices(
+                schema,
+                &self.input_buffer,
+                probe_batch,
+                build_indices,
+                probe_indices,
+                column_indices,
+                self.build_side,
+            )
+            .map(Some)
+        }
+    }
+
+    /// This function produces unmatched record results based on the build side,
+    /// join type and other parameters.
+    ///
+    /// The method uses first `prune_length` rows from the build side input buffer
+    /// to produce results.
+    ///
+    /// # Arguments
+    ///
+    /// * `output_schema` - The schema of the final output record batch.
+    /// * `prune_length` - The length of the determined prune length.
+    /// * `probe_schema` - The schema of the probe [RecordBatch].
+    /// * `join_type` - The type of join to be performed.
+    /// * `column_indices` - Indices of columns that are being joined.
+    ///
+    /// # Returns
+    ///
+    /// * `Option<RecordBatch>` - The final output record batch if required, otherwise [None].
+    fn build_side_determined_results(
+        &self,
+        output_schema: &SchemaRef,
+        prune_length: usize,
+        probe_schema: SchemaRef,
+        join_type: JoinType,
+        column_indices: &[ColumnIndex],
+    ) -> Result<Option<RecordBatch>> {
+        // Check if we need to produce a result in the final output:
+        if need_to_produce_result_in_final(self.build_side, join_type) {
+            // Calculate the indices for build and probe sides based on join type and build side:
+            let (build_indices, probe_indices) = calculate_indices_by_join_type(
+                self.build_side,
+                prune_length,
+                &self.visited_rows,
+                self.deleted_offset,
+                join_type,
+            )?;
+
+            // Create an empty probe record batch:
+            let empty_probe_batch = RecordBatch::new_empty(probe_schema);
+            // Build the final result from the indices of build and probe sides:
+            build_batch_from_indices(
+                output_schema.as_ref(),
+                &self.input_buffer,
+                &empty_probe_batch,
+                build_indices,
+                probe_indices,
+                column_indices,
+                self.build_side,
+            )
+            .map(Some)
+        } else {
+            // If we don't need to produce a result, return None
+            Ok(None)
+        }
+    }
+
+    /// Prunes the internal buffer.
+    ///
+    /// Argument `probe_batch` is used to update the intervals of the sorted
+    /// filter expressions. The updated build interval determines the new length
+    /// of the build side. If there are rows to prune, they are removed from the
+    /// internal buffer.
+    ///
+    /// # Arguments
+    ///
+    /// * `schema` - The schema of the final output record batch
+    /// * `probe_batch` - Incoming RecordBatch of the probe side.
+    /// * `probe_side_sorted_filter_expr` - Probe side mutable sorted filter expression.
+    /// * `join_type` - The type of join (e.g. inner, left, right, etc.).
+    /// * `column_indices` - A vector of column indices that specifies which columns from the
+    ///     build side should be included in the output.
+    /// * `physical_expr_graph` - A mutable reference to the physical expression graph.
+    ///
+    /// # Returns
+    ///
+    /// If there are rows to prune, returns the pruned build side record batch wrapped in an `Ok` variant.
+    /// Otherwise, returns `Ok(None)`.
+    fn prune_with_probe_batch(
+        &mut self,
+        schema: &SchemaRef,
+        probe_batch: &RecordBatch,
+        probe_side_sorted_filter_expr: &mut SortedFilterExpr,
+        join_type: JoinType,
+        column_indices: &[ColumnIndex],
+        physical_expr_graph: &mut ExprIntervalGraph,
+    ) -> Result<Option<RecordBatch>> {
+        // Check if the input buffer is empty:
+        if self.input_buffer.num_rows() == 0 {
+            return Ok(None);
+        }
+        // Convert the sorted filter expressions into a vector of (node_index, interval)
+        // tuples for use when updating the interval graph.
+        let mut filter_intervals = vec![
+            (
+                self.sorted_filter_expr.node_index(),
+                self.sorted_filter_expr.interval().clone(),
+            ),
+            (
+                probe_side_sorted_filter_expr.node_index(),
+                probe_side_sorted_filter_expr.interval().clone(),
+            ),
+        ];
+        // Use the join filter intervals to update the physical expression graph:
+        physical_expr_graph.update_ranges(&mut filter_intervals)?;
+        // Get the new join filter interval for build side:
+        let calculated_build_side_interval = filter_intervals.remove(0).1;
+        // Check if the intervals changed, exit early if not:
+        if calculated_build_side_interval.eq(self.sorted_filter_expr.interval()) {
+            return Ok(None);
+        }
+        // Determine the pruning length if there was a change in the intervals:
+        self.sorted_filter_expr
+            .set_interval(calculated_build_side_interval);
+        let prune_length =
+            determine_prune_length(&self.input_buffer, &self.sorted_filter_expr)?;
+        // If we can not prune, exit early:
+        if prune_length == 0 {
+            return Ok(None);
+        }
+        // Compute the result, and perform pruning if there are rows to prune:
+        let result = self.build_side_determined_results(
+            schema,
+            prune_length,
+            probe_batch.schema(),
+            join_type,
+            column_indices,
+        );
+        prune_hash_values(
+            prune_length,
+            &mut self.hashmap,
+            &mut self.row_hash_values,
+            self.deleted_offset as u64,
+        )?;
+        for row in self.deleted_offset..(self.deleted_offset + prune_length) {
+            self.visited_rows.remove(&row);
+        }
+        self.input_buffer = self
+            .input_buffer
+            .slice(prune_length, self.input_buffer.num_rows() - prune_length);
+        self.deleted_offset += prune_length;
+        result
+    }
+}
+
+fn combine_two_batches(
+    output_schema: &SchemaRef,
+    left_batch: Option<RecordBatch>,
+    right_batch: Option<RecordBatch>,
+) -> Result<Option<RecordBatch>> {
+    match (left_batch, right_batch) {
+        (Some(batch), None) | (None, Some(batch)) => {
+            // If only one of the batches are present, return it:
+            Ok(Some(batch))
+        }
+        (Some(left_batch), Some(right_batch)) => {
+            // If both batches are present, concatenate them:
+            concat_batches(output_schema, &[left_batch, right_batch])
+                .map_err(DataFusionError::ArrowError)
+                .map(Some)
+        }
+        (None, None) => {
+            // If neither is present, return an empty batch:
+            Ok(None)
+        }
+    }
+}
+
+impl SymmetricHashJoinStream {
+    /// Polls the next result of the join operation.
+    ///
+    /// If the result of the join is ready, it returns the next record batch.
+    /// If the join has completed and there are no more results, it returns
+    /// `Poll::Ready(None)`. If the join operation is not complete, but the
+    /// current stream is not ready yet, it returns `Poll::Pending`.
+    fn poll_next_impl(
+        &mut self,
+        cx: &mut std::task::Context<'_>,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        loop {
+            // If the final result has already been obtained, return `Poll::Ready(None)`:
+            if self.final_result {
+                return Poll::Ready(None);
+            }
+            // If both streams have been exhausted, return the final result:
+            if self.right.exhausted && self.left.exhausted {
+                // Get left side results:
+                let left_result = self.left.build_side_determined_results(
+                    &self.schema,
+                    self.left.input_buffer.num_rows(),
+                    self.right.input_buffer.schema(),
+                    self.join_type,
+                    &self.column_indices,
+                )?;
+                // Get right side results:
+                let right_result = self.right.build_side_determined_results(
+                    &self.schema,
+                    self.right.input_buffer.num_rows(),
+                    self.left.input_buffer.schema(),
+                    self.join_type,
+                    &self.column_indices,
+                )?;
+                self.final_result = true;
+                // Combine results:
+                let result =
+                    combine_two_batches(&self.schema, left_result, right_result)?;
+                // Update the metrics if we have a batch; otherwise, continue the loop.
+                if let Some(batch) = &result {
+                    self.metrics.output_batches.add(1);
+                    self.metrics.output_rows.add(batch.num_rows());
+                    return Poll::Ready(Ok(result).transpose());
+                } else {
+                    continue;
+                }
+            }
+
+            // Determine which stream should be polled next. The side the
+            // RecordBatch comes from becomes the probe side.
+            let (
+                input_stream,
+                probe_hash_joiner,
+                build_hash_joiner,
+                build_join_side,
+                probe_side_metrics,
+            ) = if self.probe_side.eq(&JoinSide::Left) {
+                (
+                    &mut self.left_stream,
+                    &mut self.left,
+                    &mut self.right,
+                    JoinSide::Right,
+                    &mut self.metrics.left,
+                )
+            } else {
+                (
+                    &mut self.right_stream,
+                    &mut self.right,
+                    &mut self.left,
+                    JoinSide::Left,
+                    &mut self.metrics.right,
+                )
+            };
+            // Poll the next batch from `input_stream`:
+            match input_stream.poll_next_unpin(cx) {
+                // Batch is available
+                Poll::Ready(Some(Ok(probe_batch))) => {
+                    // Update the metrics for the stream that was polled:
+                    probe_side_metrics.input_batches.add(1);
+                    probe_side_metrics.input_rows.add(probe_batch.num_rows());
+                    // Update the internal state of the hash joiner for the build side:
+                    probe_hash_joiner
+                        .update_internal_state(&probe_batch, &self.random_state)?;
+                    // Calculate filter intervals:
+                    calculate_filter_expr_intervals(
+                        &build_hash_joiner.input_buffer,
+                        &mut build_hash_joiner.sorted_filter_expr,
+                        &probe_batch,
+                        &mut probe_hash_joiner.sorted_filter_expr,
+                    )?;
+                    // Join the two sides:
+                    let equal_result = build_hash_joiner.join_with_probe_batch(
+                        &self.schema,
+                        self.join_type,
+                        &probe_hash_joiner.on,
+                        &self.filter,
+                        &probe_batch,
+                        &mut probe_hash_joiner.visited_rows,
+                        probe_hash_joiner.offset,
+                        &self.column_indices,
+                        &self.random_state,
+                        &self.null_equals_null,
+                    )?;
+                    // Increment the offset for the probe hash joiner:
+                    probe_hash_joiner.offset += probe_batch.num_rows();
+                    // Prune the build side input buffer using the expression
+                    // DAG and filter intervals:
+                    let anti_result = build_hash_joiner.prune_with_probe_batch(
+                        &self.schema,
+                        &probe_batch,
+                        &mut probe_hash_joiner.sorted_filter_expr,
+                        self.join_type,
+                        &self.column_indices,
+                        &mut self.physical_expr_graph,
+                    )?;
+                    // Combine results:
+                    let result =
+                        combine_two_batches(&self.schema, equal_result, anti_result)?;
+                    // Choose next poll side. If the other side is not exhausted,
+                    // switch the probe side before returning the result.
+                    if !build_hash_joiner.exhausted {
+                        self.probe_side = build_join_side;
+                    }
+                    // Update the metrics if we have a batch; otherwise, continue the loop.
+                    if let Some(batch) = &result {
+                        self.metrics.output_batches.add(1);
+                        self.metrics.output_rows.add(batch.num_rows());
+                        return Poll::Ready(Ok(result).transpose());
+                    }
+                }
+                Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))),
+                Poll::Ready(None) => {
+                    // Mark the probe side exhausted:
+                    probe_hash_joiner.exhausted = true;
+                    // Change the probe side:
+                    self.probe_side = build_join_side;
+                }
+                Poll::Pending => {
+                    if !build_hash_joiner.exhausted {
+                        self.probe_side = build_join_side;
+                    } else {
+                        return Poll::Pending;
+                    }
+                }
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use std::fs::File;
+
+    use arrow::array::{Array, ArrayRef};
+    use arrow::array::{Int32Array, TimestampNanosecondArray};
+    use arrow::compute::SortOptions;
+    use arrow::datatypes::{DataType, Field, Schema};
+    use arrow::util::pretty::pretty_format_batches;
+    use rstest::*;
+    use tempfile::TempDir;
+
+    use datafusion_expr::Operator;
+    use datafusion_physical_expr::expressions::{BinaryExpr, Column};
+    use datafusion_physical_expr::intervals::test_utils::gen_conjunctive_numeric_expr;
+    use datafusion_physical_expr::PhysicalExpr;
+
+    use crate::physical_plan::joins::{
+        hash_join_utils::tests::complicated_filter, HashJoinExec, PartitionMode,
+    };
+    use crate::physical_plan::{
+        collect, common, memory::MemoryExec, repartition::RepartitionExec,
+    };
+    use crate::prelude::{SessionConfig, SessionContext};
+    use crate::test_util;
+
+    use super::*;
+
+    const TABLE_SIZE: i32 = 1_000;
+
+    fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) {
+        // compare
+        let first_formatted = pretty_format_batches(collected_1).unwrap().to_string();
+        let second_formatted = pretty_format_batches(collected_2).unwrap().to_string();
+
+        let mut first_formatted_sorted: Vec<&str> =
+            first_formatted.trim().lines().collect();
+        first_formatted_sorted.sort_unstable();
+
+        let mut second_formatted_sorted: Vec<&str> =
+            second_formatted.trim().lines().collect();
+        second_formatted_sorted.sort_unstable();
+
+        for (i, (first_line, second_line)) in first_formatted_sorted
+            .iter()
+            .zip(&second_formatted_sorted)
+            .enumerate()
+        {
+            assert_eq!((i, first_line), (i, second_line));
+        }
+    }
+    #[allow(clippy::too_many_arguments)]
+    async fn partitioned_sym_join_with_filter(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: bool,
+        context: Arc<TaskContext>,
+    ) -> Result<Vec<RecordBatch>> {
+        let partition_count = 4;
+
+        let left_expr = on
+            .iter()
+            .map(|(l, _)| Arc::new(l.clone()) as _)
+            .collect::<Vec<_>>();
+
+        let right_expr = on
+            .iter()
+            .map(|(_, r)| Arc::new(r.clone()) as _)
+            .collect::<Vec<_>>();
+
+        let join = SymmetricHashJoinExec::try_new(
+            Arc::new(RepartitionExec::try_new(
+                left,
+                Partitioning::Hash(left_expr, partition_count),
+            )?),
+            Arc::new(RepartitionExec::try_new(
+                right,
+                Partitioning::Hash(right_expr, partition_count),
+            )?),
+            on,
+            filter,
+            join_type,
+            &null_equals_null,
+        )?;
+
+        let mut batches = vec![];
+        for i in 0..partition_count {
+            let stream = join.execute(i, context.clone())?;
+            let more_batches = common::collect(stream).await?;
+            batches.extend(
+                more_batches
+                    .into_iter()
+                    .filter(|b| b.num_rows() > 0)
+                    .collect::<Vec<_>>(),
+            );
+        }
+
+        Ok(batches)
+    }
+    #[allow(clippy::too_many_arguments)]
+    async fn partitioned_hash_join_with_filter(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: bool,
+        context: Arc<TaskContext>,
+    ) -> Result<Vec<RecordBatch>> {
+        let partition_count = 4;
+
+        let (left_expr, right_expr) = on
+            .iter()
+            .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
+            .unzip();
+
+        let join = HashJoinExec::try_new(
+            Arc::new(RepartitionExec::try_new(
+                left,
+                Partitioning::Hash(left_expr, partition_count),
+            )?),
+            Arc::new(RepartitionExec::try_new(
+                right,
+                Partitioning::Hash(right_expr, partition_count),
+            )?),
+            on,
+            Some(filter),
+            join_type,
+            PartitionMode::Partitioned,
+            &null_equals_null,
+        )?;
+
+        let mut batches = vec![];
+        for i in 0..partition_count {
+            let stream = join.execute(i, context.clone())?;
+            let more_batches = common::collect(stream).await?;
+            batches.extend(
+                more_batches
+                    .into_iter()
+                    .filter(|b| b.num_rows() > 0)
+                    .collect::<Vec<_>>(),
+            );
+        }
+
+        Ok(batches)
+    }
+
+    pub fn split_record_batches(
+        batch: &RecordBatch,
+        batch_size: usize,
+    ) -> Result<Vec<RecordBatch>> {
+        let row_num = batch.num_rows();
+        let number_of_batch = row_num / batch_size;
+        let mut sizes = vec![batch_size; number_of_batch];
+        sizes.push(row_num - (batch_size * number_of_batch));
+        let mut result = vec![];
+        for (i, size) in sizes.iter().enumerate() {
+            result.push(batch.slice(i * batch_size, *size));
+        }
+        Ok(result)
+    }
+
+    fn build_record_batch(columns: Vec<(&str, ArrayRef)>) -> Result<RecordBatch> {

Review Comment:
   Yeah, it is. Resolved πŸ˜„



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1448915076

   Thanks a lot for all the work @metesynnada  and @ozankabak 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119624096


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   Sure, you can do all code refinement in following PR. 
   BTW, in your use case, do you use Ballista as the distributed engine ?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1447096516

   Marking as draft as this one is waiting on changes so it doesn't appear on the "ready for review" list: https://github.com/apache/arrow-datafusion/pulls?q=is%3Apr+is%3Aopen+-is%3Adraft+


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] Dandandan commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "Dandandan (via GitHub)" <gi...@apache.org>.
Dandandan commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120208149


##########
datafusion/common/src/scalar.rs:
##########
@@ -1019,6 +1019,29 @@ impl ScalarValue {
         Self::List(scalars, Box::new(Field::new("item", child_type, true)))
     }
 
+    // Create a zero value in the given type.
+    pub fn new_zero(datatype: &DataType) -> Result<ScalarValue> {
+        assert!(datatype.is_primitive());
+        Ok(match datatype {
+            DataType::Boolean => ScalarValue::Boolean(Some(false)),
+            DataType::Int8 => ScalarValue::Int8(Some(0)),
+            DataType::Int16 => ScalarValue::Int16(Some(0)),
+            DataType::Int32 => ScalarValue::Int32(Some(0)),
+            DataType::Int64 => ScalarValue::Int64(Some(0)),
+            DataType::UInt8 => ScalarValue::UInt8(Some(0)),
+            DataType::UInt16 => ScalarValue::UInt16(Some(0)),
+            DataType::UInt32 => ScalarValue::UInt32(Some(0)),
+            DataType::UInt64 => ScalarValue::UInt64(Some(0)),
+            DataType::Float32 => ScalarValue::UInt64(Some(0)),

Review Comment:
   This should be float32 and float64?



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115712579


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   makes sense



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115295815


##########
datafusion/physical-expr/src/intervals/mod.rs:
##########
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   At the moment, the current implementations are somewhat limited in their ability to handle complex expressions, and it is challenging to expand them beyond their current capabilities. However, with the maturation of the interval arithmetics library, we can explore more advanced expression analysis, which holds the potential for significant progress.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1442621079

   > This sounds good to me too. We will need to do this at some point, physical plan code is getting quite big.
   
   I believe the issue was that that last time we tried a bunch of SessionContext and the traits it used (like TableProvider) were all needed so there was no clean way to split out the physical plans
   
   Now that @tustvold  made a harder split between SessionState and SessionContext it may be more tractable now. 
   
   I'll plan to file / record the follow on items as tickets prior to merging this PR


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1442975303

   This is really a great and exciting feature! 


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1438468030

   >  It may be possible to do this if we carve out things like interval arithmetic, constraint propagation etc. into separate PRs. However, that will deprive you of important context (i.e. where/how do we use these things and why do they have the APIs that they have). For this reason, I suggest keeping this in one piece, and following a top-down order to make reviewing easier (and in parts):
   
   I agree it is a tradeoff. Your suggestion of reviewing as one PR sounds good.
   
   > I agree, it comes down to personal work style. Let's play by ear and see if my suggestions above help. If not, we can always try breaking it up. Our goal is to make the review process be efficient/easy, after all πŸ™‚
   
   Yes, thank you! I am just trying to offer suggestions to help get your PRs in faster in the future. 
   
   (I am reading the blog post this morning so hopefully will be ready to review more in depth the next few days)
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119865212


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   Good to know.  In the past, there was some discussion to enhance Ballista to support both BATCH/STREAMING execution models:
   
   https://docs.google.com/document/d/1OdAe078axk4qO0ozUxNqBMD4wKoBhzh9keMuLp_jerE/edit#
   http://www.vldb.org/pvldb/vol11/p746-yin.pdf
   I haven't working on Ballista since last year and there is no progress in this area.
   
   And in the latest Flink release, they had implement similar features(Bubble execution model, hybird shuffle etc).
   https://cwiki.apache.org/confluence/display/FLINK/FLIP-235%3A+Hybrid+Shuffle+Mode
   
   I think generally we can follow Flink's approach to make both DataFusion and Ballista support BATCH/STREAMING execution models.  In the high level, we can have different models(BATCH vs STREAMING), and user can specify the execution model. In the physical planing phase, we have `BatchPlanner` and `StreamingPlanner`, they can share some common rules, and batch and streaming planners can have their own rules. 
   In the `ExecutionPlan` trait, we can have another trait to indicate some operators are `Source` operators, the source operators can be `BOUNDED` or `UNBOUNDED`.    `BOUNDED` or `UNBOUNDED` should be a property available to
   Source operators only. 
   
   



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119605173


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   We will think about this. I don't see a straightforward way right away but I will talk to @metesynnada about it in detail. If this turns out to be possible, we can make a follow-on PR about 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada closed pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada closed pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)
URL: https://github.com/apache/arrow-datafusion/pull/5322


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116467854


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {

Review Comment:
   Yes, please add some comments, specifically for the `data` member.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114957696


##########
datafusion/physical-expr/src/intervals/mod.rs:
##########
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   We evaluated the existing approaches but couldn't leverage them for this as they were not general enough (and we didn't see a way to generalize them in their existing structure). @metesynnada can provide more details about this.
   
   > Would you be willing to help unify the implementations (I don't really have a preference as to which, only that we don't have three)
   
   Yes! This is actually on our roadmap, and we are designing the interval arithmetic library so that it is general purpose and appropriate for other use cases too.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114945924


##########
datafusion/core/src/execution/context.rs:
##########
@@ -1528,6 +1528,9 @@ impl SessionState {
             // repartitioning and local sorting steps to meet distribution and ordering requirements.
             // Therefore, it should run before EnforceDistribution and EnforceSorting.
             Arc::new(JoinSelection::new()),
+            // Enforce sort before PipelineFixer

Review Comment:
   We haven't focused on optimal rule ordering yet. Right now, we only enable SHJ when joining two unbounded streams, so it has to happen at some point before `PipelineFixer`. As we mature the SHJ implementation, we will enable it even for normal tables when it is appropriate (yields performance gains). Within that context, we will revisit rule ordering.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116776024


##########
datafusion/core/src/physical_plan/joins/hash_join_utils.rs:
##########
@@ -0,0 +1,598 @@
+// 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 contains common subroutines for regular and symmetric hash join
+//! related functionality, used both in join calculations and optimization rules.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+use std::usize;
+
+use arrow::datatypes::SchemaRef;
+
+use datafusion_common::DataFusionError;
+use datafusion_physical_expr::expressions::Column;
+use datafusion_physical_expr::intervals::Interval;
+use datafusion_physical_expr::rewrite::TreeNodeRewritable;
+use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr};
+
+use crate::common::Result;
+use crate::physical_plan::joins::utils::{JoinFilter, JoinSide};
+
+fn check_filter_expr_contains_sort_information(
+    expr: &Arc<dyn PhysicalExpr>,
+    reference: &Arc<dyn PhysicalExpr>,
+) -> bool {
+    expr.eq(reference)
+        || expr
+            .children()
+            .iter()
+            .any(|e| check_filter_expr_contains_sort_information(e, reference))
+}
+
+fn collect_columns_recursive(expr: &Arc<dyn PhysicalExpr>, columns: &mut Vec<Column>) {
+    if let Some(column) = expr.as_any().downcast_ref::<Column>() {
+        if !columns.iter().any(|c| c.eq(column)) {
+            columns.push(column.clone())
+        }
+    }
+    expr.children()
+        .iter()
+        .for_each(|e| collect_columns_recursive(e, columns))
+}
+
+fn collect_columns(expr: &Arc<dyn PhysicalExpr>) -> Vec<Column> {
+    let mut columns = vec![];
+    collect_columns_recursive(expr, &mut columns);
+    columns
+}
+
+/// Create a one to one mapping from main columns to filter columns using
+/// filter column indices. A column index looks like:
+/// ```text
+/// ColumnIndex {
+///     index: 0, // field index in main schema
+///     side: JoinSide::Left, // child side
+/// }
+/// ```
+pub fn map_origin_col_to_filter_col(
+    filter: &JoinFilter,
+    schema: &SchemaRef,
+    side: &JoinSide,
+) -> Result<HashMap<Column, Column>> {
+    let filter_schema = filter.schema();
+    let mut col_to_col_map: HashMap<Column, Column> = HashMap::new();
+    for (filter_schema_index, index) in filter.column_indices().iter().enumerate() {
+        if index.side.eq(side) {
+            // Get the main field from column index:
+            let main_field = schema.field(index.index);
+            // Create a column expression:
+            let main_col = Column::new_with_schema(main_field.name(), schema.as_ref())?;
+            // Since the order of by filter.column_indices() is the same with
+            // that of intermediate schema fields, we can get the column directly.
+            let filter_field = filter_schema.field(filter_schema_index);
+            let filter_col = Column::new(filter_field.name(), filter_schema_index);
+            // Insert mapping:
+            col_to_col_map.insert(main_col, filter_col);
+        }
+    }
+    Ok(col_to_col_map)
+}
+
+/// This function analyzes [PhysicalSortExpr] graphs with respect to monotonicity
+/// (sorting) properties. This is necessary since monotonically increasing and/or
+/// decreasing expressions are required when using join filter expressions for
+/// data pruning purposes.
+///
+/// The method works as follows:
+/// 1. Maps the original columns to the filter columns using the `map_origin_col_to_filter_col` function.
+/// 2. Collects all columns in the sort expression using the `PhysicalExprColumnCollector` visitor.
+/// 3. Checks if all columns are included in the `column_mapping_information` map.
+/// 4. If all columns are included, the sort expression is converted into a filter expression using the `transform_up` and `convert_filter_columns` functions.
+/// 5. Searches the converted filter expression in the filter expression using the `check_filter_expr_contains_sort_information`.
+/// 6. If an exact match is encountered, returns the converted filter expression as `Some(Arc<dyn PhysicalExpr>)`.
+/// 7. If all columns are not included or the exact match is not encountered, returns `None`.
+///
+/// Examples:
+/// Consider the filter expression "a + b > c + 10 AND a + b < c + 100".
+/// 1. If the expression "a@ + d@" is sorted, it will not be accepted since the "d@" column is not part of the filter.
+/// 2. If the expression "d@" is sorted, it will not be accepted since the "d@" column is not part of the filter.
+/// 3. If the expression "a@ + b@ + c@" is sorted, all columns are represented in the filter expression. However,
+///    there is no exact match, so this expression does not indicate pruning.
+pub fn convert_sort_expr_with_filter_schema(
+    side: &JoinSide,
+    filter: &JoinFilter,
+    schema: &SchemaRef,
+    sort_expr: &PhysicalSortExpr,
+) -> Result<Option<Arc<dyn PhysicalExpr>>> {

Review Comment:
   I think you can also put `EquivalenceProperties` into the consideration, you can do this in the following PR.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1116780755


##########
datafusion/core/src/physical_plan/joins/utils.rs:
##########
@@ -770,52 +794,49 @@ pub(crate) fn apply_join_filter_to_indices(
     let mask = as_boolean_array(&filter_result)?;
 
     let left_filtered = PrimitiveArray::<UInt64Type>::from(
-        compute::filter(&left_indices, mask)?.data().clone(),
+        compute::filter(&build_indices, mask)?.data().clone(),
     );
     let right_filtered = PrimitiveArray::<UInt32Type>::from(
-        compute::filter(&right_indices, mask)?.data().clone(),
+        compute::filter(&probe_indices, mask)?.data().clone(),
     );
-
     Ok((left_filtered, right_filtered))
 }
 
 /// Returns a new [RecordBatch] by combining the `left` and `right` according to `indices`.
 /// The resulting batch has [Schema] `schema`.
 pub(crate) fn build_batch_from_indices(
     schema: &Schema,
-    left: &RecordBatch,
-    right: &RecordBatch,
-    left_indices: UInt64Array,
-    right_indices: UInt32Array,
+    build_input_buffer: &RecordBatch,

Review Comment:
   πŸ‘



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441298859

   In the past, we had implement something called Range Join which internally build some range indexes, I am not familiar with Symmetric Hash Join, I will take a closer look at this PR.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1114333530


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {

Review Comment:
   Can you please add some documentation to this structure explaining its use?



##########
datafusion/core/src/execution/context.rs:
##########
@@ -1528,6 +1528,9 @@ impl SessionState {
             // repartitioning and local sorting steps to meet distribution and ordering requirements.
             // Therefore, it should run before EnforceDistribution and EnforceSorting.
             Arc::new(JoinSelection::new()),
+            // Enforce sort before PipelineFixer

Review Comment:
   Why doess the enforcement happens before pipeline fixer (and not for example, also before `JoinSelection`)?



##########
datafusion/physical-expr/src/intervals/test_utils.rs:
##########
@@ -0,0 +1,67 @@
+// 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.
+
+//! Test utilities for the interval arithmetic library
+
+use std::sync::Arc;
+
+use crate::expressions::{BinaryExpr, Literal};
+use crate::PhysicalExpr;
+use datafusion_common::ScalarValue;
+use datafusion_expr::Operator;
+
+#[allow(clippy::too_many_arguments)]
+/// This test function generates a conjunctive statement with two numeric
+/// terms with the following form:
+/// left_col (op_1) a  > right_col (op_2) b AND left_col (op_3) c < right_col (op_4) d
+pub fn gen_conjunctive_numeric_expr(
+    left_col: Arc<dyn PhysicalExpr>,
+    right_col: Arc<dyn PhysicalExpr>,
+    op_1: Operator,
+    op_2: Operator,
+    op_3: Operator,
+    op_4: Operator,
+    a: i32,
+    b: i32,
+    c: i32,
+    d: i32,
+) -> Arc<dyn PhysicalExpr> {
+    let left_and_1 = Arc::new(BinaryExpr::new(
+        left_col.clone(),
+        op_1,
+        Arc::new(Literal::new(ScalarValue::Int32(Some(a)))),

Review Comment:
   FWIW I think you can use the `lit` function to make this code easier to write:
   
   https://docs.rs/datafusion-physical-expr/18.0.0/datafusion_physical_expr/expressions/fn.lit.html
   
   ```suggestion
           Arc::new(lit(a)),
   ```



##########
datafusion/physical-expr/Cargo.toml:
##########
@@ -60,6 +60,7 @@ lazy_static = { version = "^1.4.0" }
 md-5 = { version = "^0.10.0", optional = true }
 num-traits = { version = "0.2", default-features = false }
 paste = "^1.0"
+petgraph = "0.6.2"

Review Comment:
   I reviewed this crate and its dependencies: https://crates.io/crates/petgraph/reverse_dependencies and it looks fine to add to me.
   
   Given we already have a dependency on parking_lot I don't think this is a net new dependency, and furthermore it seems widely used across the Rust ecosystem. 



##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -77,6 +87,104 @@ impl PhysicalOptimizerRule for PipelineFixer {
     }
 }
 
+/// Indicates whether interval arithmetic is supported for the given expression.
+/// Currently, we do not support all [PhysicalExpr]s for interval calculations.
+/// We do not support every type of [Operator]s either. Over time, this check
+/// will relax as more types of [PhysicalExpr]s and [Operator]s are supported.
+/// Currently, [CastExpr], [BinaryExpr], [Column] and [Literal] is supported.
+fn check_support(expr: &Arc<dyn PhysicalExpr>) -> bool {
+    let expr_any = expr.as_any();
+    let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::<BinaryExpr>()
+    {
+        is_operator_supported(binary_expr.op())
+    } else {
+        expr_any.is::<Column>() || expr_any.is::<Literal>() || expr_any.is::<CastExpr>()
+    };
+    expr_supported && expr.children().iter().all(check_support)
+}
+
+/// This function returns whether a given hash join is replaceable by a
+/// symmetric hash join. Basically, the requirement is that involved
+/// [PhysicalExpr]s, [Operator]s and data types need to be supported,
+/// and order information must cover every column in the filter expression.
+fn is_suitable_for_symmetric_hash_join(hash_join: &HashJoinExec) -> Result<bool> {
+    if let Some(filter) = hash_join.filter() {
+        let left = hash_join.left();
+        if let Some(left_ordering) = left.output_ordering() {
+            let right = hash_join.right();
+            if let Some(right_ordering) = right.output_ordering() {
+                let expr_supported = check_support(filter.expression());
+                let left_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Left,
+                    filter,
+                    &left.schema(),
+                    &left_ordering[0],
+                )?
+                .is_some();
+                let right_convertible = convert_sort_expr_with_filter_schema(
+                    &JoinSide::Right,
+                    filter,
+                    &right.schema(),
+                    &right_ordering[0],
+                )?
+                .is_some();
+                let fields_supported = filter
+                    .schema()
+                    .fields()
+                    .iter()
+                    .all(|f| is_datatype_supported(f.data_type()));
+                return Ok(expr_supported
+                    && fields_supported
+                    && left_convertible
+                    && right_convertible);
+            }
+        }
+    }
+    Ok(false)
+}
+
+/// This subrule checks if one can replace a hash join with a symmetric hash
+/// join so that the pipeline does not break due to the join operation in
+/// question. If possible, it makes this replacement; otherwise, it has no
+/// effect.
+fn hash_join_convert_symmetric_subrule(
+    input: PipelineStatePropagator,
+) -> Option<Result<PipelineStatePropagator>> {
+    let plan = input.plan;
+    if let Some(hash_join) = plan.as_any().downcast_ref::<HashJoinExec>() {
+        let ub_flags = input.children_unbounded;
+        let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]);

Review Comment:
   I wonder why this PR limits the use of symmetric hash join to unbounded streams? It seems it could be used for any input where the sort conditions are reasonable, given
   
   1. the very good numbers reported in https://synnada.notion.site/synnada/General-purpose-Stream-Joins-via-Pruning-Symmetric-Hash-Joins-2fe26d3127a241e294a0217b1f18603a 
   2. the fact that this algorithm is general for any sorted stream (not just the unbounded ones)



##########
datafusion/physical-expr/src/intervals/mod.rs:
##########
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   This is a neat approach; However, I am concerned that with it we wll have three partially complete range analysis implementations:
   
   # Range analysis used in cardinality estimation 
   https://docs.rs/datafusion-physical-expr/18.0.0/datafusion_physical_expr/trait.PhysicalExpr.html#method.analyze
   https://docs.rs/datafusion-physical-expr/18.0.0/datafusion_physical_expr/struct.AnalysisContext.html
   (cc @isidentical)
   
   # Pruning Predicate
   https://docs.rs/datafusion/18.0.0/datafusion/physical_optimizer/pruning/struct.PruningPredicate.html
   
   The modules are all structured a differently / have different implementations but what they are computing (interval / range analysis) is the same
   
   Did you consider the other implementations? Would you be willing to help unify the implementations (I don't really have a preference as to which, only that we don't have three)



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115291450


##########
datafusion/core/src/execution/context.rs:
##########
@@ -1528,6 +1528,9 @@ impl SessionState {
             // repartitioning and local sorting steps to meet distribution and ordering requirements.
             // Therefore, it should run before EnforceDistribution and EnforceSorting.
             Arc::new(JoinSelection::new()),
+            // Enforce sort before PipelineFixer

Review Comment:
   If the filter expression is incapable of pruning both sides, bounded sources will not experience any performance boost since there is no sliding window effect. We plan to include a check for whether the filter expression supports range pruning on both sides in a future PR. Once this check is in place, we can evaluate the feasibility of applying this plan to bounded sources.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441840741

   >  In the past, we had implement something called Range Join which internally build some range indexes, I am not familiar with Symmetric Hash Join, I will take a closer look at this PR.
   
   I believe SHJ handles a larger class of queries as a typical Range Join query, though I suspect that given all else being equal one could probably make range joins go faster as there are typically no hash tables to maintain
   
   The excellent blog post from this PR has great background on symmetric hash join https://www.synnada.ai/blog/general-purpose-stream-joins-via-pruning-symmetric-hash-joins
   
   More reading on range joins https://www.vertica.com/blog/what-is-a-range-join-and-why-is-it-so-fastba-p223413/
   
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1443017531

   > I read the blog https://synnada.notion.site/synnada/General-purpose-Stream-Joins-via-Pruning-Symmetric-Hash-Joins-2fe26d3127a241e294a0217b1f18603a which was very helpful understanding this PR. Thank you so much for writing that.
   > 
   > I made it about half way though this PR today and hope to complete my initial review tomorrow but wanted to leave the feedback I had so far. .
   > 
   > I initially focused on the larger architectural structure and I found this code very well commented and tested ❀️
   > 
   > My biggest concern so far is the addition of a third way to do interval analysis. I really like the rigor and sophistication of the interval analysis in this PR but I worry about having so many.
   > 
   > All in all really impressive work @metesynnada
   
   really nice tech blog!!


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115463078


##########
datafusion/physical-expr/src/intervals/mod.rs:
##########
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Yes, really hope we could unify the Range/Interval analysis framework into one place.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1115744932


##########
datafusion/physical-expr/src/intervals/interval_aritmetic.rs:
##########
@@ -0,0 +1,533 @@
+// 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.
+
+//! Interval arithmetic library
+
+use std::borrow::Borrow;
+use std::fmt;
+use std::fmt::{Display, Formatter};
+
+use arrow::compute::{cast_with_options, CastOptions};
+use arrow::datatypes::DataType;
+use datafusion_common::{DataFusionError, Result, ScalarValue};
+use datafusion_expr::Operator;
+
+use crate::aggregate::min_max::{max, min};
+
+/// This type represents an interval, which is used to calculate reliable
+/// bounds for expressions. Currently, we only support addition and
+/// subtraction, but more capabilities will be added in the future.
+/// Upper/lower bounds having NULL values indicate an unbounded side. For
+/// example; [10, 20], [10, ∞], [-∞, 100] and [-∞, ∞] are all valid intervals.
+#[derive(Debug, PartialEq, Clone, Eq, Hash)]
+pub struct Interval {
+    pub lower: ScalarValue,
+    pub upper: ScalarValue,

Review Comment:
   Longer term to support partition pruning, supporting Interval arithmetic on `Array`s rather than `ScalarValue`s might be good / lower the cost to evaluate them. 



##########
datafusion/core/src/physical_plan/memory.rs:
##########
@@ -144,6 +146,25 @@ impl MemoryExec {
             schema,
             projected_schema,
             projection,
+            sort_information: None,
+        })
+    }
+    /// Create a new execution plan for reading in-memory record batches
+    /// The provided `schema` should not have the projection applied. Also, you can specify sort
+    /// information on PhysicalExprs.
+    pub fn try_new_with_sort_information(

Review Comment:
   FWIW another pattern that might be nice would be a builder style:
   
   ```rust
       pub fn with_sort_information(mut self, sort_information: Option<Vec<PhysicalSortExpr>>) -> Self {
     ...
   }
   ```
   
   so that then the creation looks like 
   ```rust
   let exec = MemExec::try_new(partitions, schema, projection)?
     .with_sort_information(sort_information)
   ```



##########
datafusion/physical-expr/src/intervals/cp_solver.rs:
##########
@@ -0,0 +1,1088 @@
+// 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.
+
+//! Constraint propagator/solver for custom PhysicalExpr graphs.
+
+use std::collections::HashSet;
+use std::fmt::{Display, Formatter};
+use std::ops::Index;
+use std::sync::Arc;
+
+use arrow_schema::DataType;
+use datafusion_common::{Result, ScalarValue};
+use datafusion_expr::Operator;
+use petgraph::graph::NodeIndex;
+use petgraph::stable_graph::{DefaultIx, StableGraph};
+use petgraph::visit::{Bfs, Dfs, DfsPostOrder, EdgeRef};
+use petgraph::Outgoing;
+
+use crate::expressions::{BinaryExpr, CastExpr, Literal};
+use crate::intervals::interval_aritmetic::{apply_operator, Interval};
+use crate::utils::{build_dag, ExprTreeNode};
+use crate::PhysicalExpr;
+
+// Interval arithmetic provides a way to perform mathematical operations on
+// intervals, which represent a range of possible values rather than a single
+// point value. This allows for the propagation of ranges through mathematical
+// operations, and can be used to compute bounds for a complicated expression.
+// The key idea is that by breaking down a complicated expression into simpler
+// terms, and then combining the bounds for those simpler terms, one can
+// obtain bounds for the overall expression.
+//
+// For example, consider a mathematical expression such as x^2 + y = 4. Since
+// it would be a binary tree in [PhysicalExpr] notation, this type of an
+// hierarchical computation is well-suited for a graph based implementation.
+// In such an implementation, an equation system f(x) = 0 is represented by a
+// directed acyclic expression graph (DAEG).
+//
+// In order to use interval arithmetic to compute bounds for this expression,
+// one would first determine intervals that represent the possible values of x
+// and y. Let's say that the interval for x is [1, 2] and the interval for y
+// is [-3, 1]. In the chart below, you can see how the computation takes place.
+//
+// This way of using interval arithmetic to compute bounds for a complex
+// expression by combining the bounds for the constituent terms within the
+// original expression allows us to reason about the range of possible values
+// of the expression. This information later can be used in range pruning of
+// the provably unnecessary parts of `RecordBatch`es.
+//
+// References
+// 1 - Kabak, Mehmet Ozan. Analog Circuit Start-Up Behavior Analysis: An Interval
+// Arithmetic Based Approach, Chapter 4. Stanford University, 2015.
+// 2 - Moore, Ramon E. Interval analysis. Vol. 4. Englewood Cliffs: Prentice-Hall, 1966.
+// 3 - F. Messine, "Deterministic global optimization using interval constraint
+// propagation techniques," RAIRO-Operations Research, vol. 38, no. 04,
+// pp. 277{293, 2004.
+//
+// ``` text

Review Comment:
   I really like this writeup / description
   
   I think it would be great to consolidate the other range analysis in the codebase around this one



##########
datafusion/core/src/physical_plan/joins/utils.rs:
##########
@@ -770,52 +794,49 @@ pub(crate) fn apply_join_filter_to_indices(
     let mask = as_boolean_array(&filter_result)?;
 
     let left_filtered = PrimitiveArray::<UInt64Type>::from(
-        compute::filter(&left_indices, mask)?.data().clone(),
+        compute::filter(&build_indices, mask)?.data().clone(),
     );
     let right_filtered = PrimitiveArray::<UInt32Type>::from(
-        compute::filter(&right_indices, mask)?.data().clone(),
+        compute::filter(&probe_indices, mask)?.data().clone(),
     );
-
     Ok((left_filtered, right_filtered))
 }
 
 /// Returns a new [RecordBatch] by combining the `left` and `right` according to `indices`.
 /// The resulting batch has [Schema] `schema`.
 pub(crate) fn build_batch_from_indices(
     schema: &Schema,
-    left: &RecordBatch,
-    right: &RecordBatch,
-    left_indices: UInt64Array,
-    right_indices: UInt32Array,
+    build_input_buffer: &RecordBatch,

Review Comment:
   I agree with @Dandandan  that these new names are a significant improvement in readability



##########
datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs:
##########
@@ -0,0 +1,2640 @@
+// 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 the symmetric hash join algorithm with range-based
+//! data pruning to join two (potentially infinite) streams.
+//!
+//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate
+//! output ordering) and produces the join output according to the given join
+//! type and other options.
+//!
+//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations
+//! for both its children.
+
+use std::collections::{HashMap, VecDeque};
+use std::fmt;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+use std::task::Poll;
+use std::vec;
+use std::{any::Any, usize};
+
+use ahash::RandomState;
+use arrow::array::{
+    ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray,
+    PrimitiveBuilder,
+};
+use arrow::compute::concat_batches;
+use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use futures::{Stream, StreamExt};
+use hashbrown::{raw::RawTable, HashSet};
+
+use datafusion_common::{utils::bisect, ScalarValue};
+use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval};
+
+use crate::error::{DataFusionError, Result};
+use crate::execution::context::TaskContext;
+use crate::logical_expr::JoinType;
+use crate::physical_plan::{
+    expressions::Column,
+    expressions::PhysicalSortExpr,
+    joins::{
+        hash_join::{build_join_indices, update_hash, JoinHashMap},
+        hash_join_utils::{build_filter_input_order, SortedFilterExpr},
+        utils::{
+            build_batch_from_indices, build_join_schema, check_join_is_valid,
+            combine_join_equivalence_properties, partitioned_join_output_partitioning,
+            ColumnIndex, JoinFilter, JoinOn, JoinSide,
+        },
+    },
+    metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet},
+    DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+/// A symmetric hash join with range conditions is when both streams are hashed on the
+/// join key and the resulting hash tables are used to join the streams.
+/// The join is considered symmetric because the hash table is built on the join keys from both
+/// streams, and the matching of rows is based on the values of the join keys in both streams.
+/// This type of join is efficient in streaming context as it allows for fast lookups in the hash
+/// table, rather than having to scan through one or both of the streams to find matching rows, also it
+/// only considers the elements from the stream that fall within a certain sliding window (w/ range conditions),
+/// making it more efficient and less likely to store stale data. This enables operating on unbounded streaming
+/// data without any memory issues.
+///
+/// For each input stream, create a hash table.
+///   - For each new [RecordBatch] in build side, hash and insert into inputs hash table. Update offsets.
+///   - Test if input is equal to a predefined set of other inputs.
+///   - If so record the visited rows. If the matched row results must be produced (INNER, LEFT), output the [RecordBatch].
+///   - Try to prune other side (probe) with new [RecordBatch].
+///   - If the join type indicates that the unmatched rows results must be produced (LEFT, FULL etc.),
+/// output the [RecordBatch] when a pruning happens or at the end of the data.
+///
+///
+/// ``` text
+///                        +-------------------------+
+///                        |                         |
+///   left stream ---------|  Left OneSideHashJoiner |---+
+///                        |                         |   |
+///                        +-------------------------+   |
+///                                                      |
+///                                                      |--------- Joined output
+///                                                      |
+///                        +-------------------------+   |
+///                        |                         |   |
+///  right stream ---------| Right OneSideHashJoiner |---+
+///                        |                         |
+///                        +-------------------------+
+///
+/// Prune build side when the new RecordBatch comes to the probe side. We utilize interval arithmetic
+/// on JoinFilter's sorted PhysicalExprs to calculate the joinable range.
+///
+///
+///               PROBE SIDE          BUILD SIDE
+///                 BUFFER              BUFFER
+///             +-------------+     +------------+
+///             |             |     |            |    Unjoinable
+///             |             |     |            |    Range
+///             |             |     |            |
+///             |             |  |---------------------------------
+///             |             |  |  |            |
+///             |             |  |  |            |
+///             |             | /   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |    Joinable
+///             |             |/    |            |    Range
+///             |             ||    |            |
+///             |+-----------+||    |            |
+///             || Record    ||     |            |
+///             || Batch     ||     |            |
+///             |+-----------+||    |            |
+///             +-------------+\    +------------+
+///                             |
+///                             \
+///                              |---------------------------------
+///
+///  This happens when range conditions are provided on sorted columns. E.g.
+///
+///        SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_time > right_time - INTERVAL 12 MINUTES AND left_time < right_time + INTERVAL 2 HOUR
+///
+/// or
+///       SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_sorted > right_sorted - 3 AND left_sorted < right_sorted + 10
+///
+/// For general purpose, in the second scenario, when the new data comes to probe side, the conditions can be used to
+/// determine a specific threshold for discarding rows from the inner buffer. For example, if the sort order the
+/// two columns ("left_sorted" and "right_sorted") are ascending (it can be different in another scenarios)
+/// and the join condition is "left_sorted > right_sorted - 3" and the latest value on the right input is 1234, meaning
+/// that the left side buffer must only keep rows where "leftTime > rightTime - 3 > 1234 - 3 > 1231" ,
+/// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending)
+/// than that can be dropped from the inner buffer.
+/// ```
+pub struct SymmetricHashJoinExec {
+    /// Left side stream
+    pub(crate) left: Arc<dyn ExecutionPlan>,
+    /// Right side stream
+    pub(crate) right: Arc<dyn ExecutionPlan>,
+    /// Set of common columns used to join on
+    pub(crate) on: Vec<(Column, Column)>,
+    /// Filters applied when finding matching rows
+    pub(crate) filter: JoinFilter,
+    /// How the join is performed
+    pub(crate) join_type: JoinType,
+    /// Order information of filter expressions
+    sorted_filter_exprs: Vec<SortedFilterExpr>,
+    /// Left required sort
+    left_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Right required sort
+    right_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Expression graph for interval calculations
+    physical_expr_graph: ExprIntervalGraph,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Shares the `RandomState` for the hashing algorithm
+    random_state: RandomState,
+    /// Execution metrics
+    metrics: ExecutionPlanMetricsSet,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    /// If null_equals_null is true, null == null else null != null
+    pub(crate) null_equals_null: bool,
+}
+
+#[derive(Debug)]
+struct SymmetricHashJoinSideMetrics {
+    /// Number of batches consumed by this operator
+    input_batches: metrics::Count,
+    /// Number of rows consumed by this operator
+    input_rows: metrics::Count,
+}
+
+/// Metrics for HashJoinExec
+#[derive(Debug)]
+struct SymmetricHashJoinMetrics {
+    /// Number of left batches/rows consumed by this operator
+    left: SymmetricHashJoinSideMetrics,
+    /// Number of right batches/rows consumed by this operator
+    right: SymmetricHashJoinSideMetrics,
+    /// Number of batches produced by this operator
+    output_batches: metrics::Count,
+    /// Number of rows produced by this operator
+    output_rows: metrics::Count,
+}
+
+impl SymmetricHashJoinMetrics {
+    pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self {
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let left = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let right = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let output_batches =
+            MetricBuilder::new(metrics).counter("output_batches", partition);
+
+        let output_rows = MetricBuilder::new(metrics).output_rows(partition);
+
+        Self {
+            left,
+            right,
+            output_batches,
+            output_rows,
+        }
+    }
+}
+
+impl SymmetricHashJoinExec {
+    /// Tries to create a new [SymmetricHashJoinExec].
+    /// # Error
+    /// This function errors when:
+    /// - It is not possible to join the left and right sides on keys `on`, or
+    /// - It fails to construct [SortedFilterExpr]s, or
+    /// - It fails to create the [ExprIntervalGraph].
+    pub fn try_new(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: &bool,
+    ) -> Result<Self> {
+        let left_schema = left.schema();
+        let right_schema = right.schema();
+
+        // Error out if no "on" contraints are given:
+        if on.is_empty() {
+            return Err(DataFusionError::Plan(
+                "On constraints in SymmetricHashJoinExec should be non-empty".to_string(),
+            ));
+        }
+
+        // Check if the join is valid with the given on constraints:
+        check_join_is_valid(&left_schema, &right_schema, &on)?;
+
+        // Build the join schema from the left and right schemas:
+        let (schema, column_indices) =
+            build_join_schema(&left_schema, &right_schema, join_type);
+
+        // Set a random state for the join:
+        let random_state = RandomState::with_seeds(0, 0, 0, 0);
+
+        // Create an expression DAG for the join filter:
+        let mut physical_expr_graph =
+            ExprIntervalGraph::try_new(filter.expression().clone())?;
+
+        // Interval calculations require each column to exhibit monotonicity
+        // independently. However, a `PhysicalSortExpr` object defines a
+        // lexicographical ordering, so we can only use their first elements.
+        // when deducing column monotonicities.
+        // TODO: Extend the `PhysicalSortExpr` mechanism to express independent
+        //       (i.e. simultaneous) ordering properties of columns.
+        let (left_ordering, right_ordering) = match (
+            left.output_ordering(),
+            right.output_ordering(),
+        ) {
+            (Some([left_ordering, ..]), Some([right_ordering, ..])) => {
+                (left_ordering, right_ordering)
+            }
+            _ => {
+                return Err(DataFusionError::Plan(
+                    "Symmetric hash join requires its children to have an output ordering".to_string(),
+                ));
+            }
+        };
+
+        // Build the sorted filter expression for the left child:
+        let left_filter_expression = build_filter_input_order(
+            JoinSide::Left,
+            &filter,
+            &left.schema(),
+            left_ordering,
+        )?;
+
+        // Build the sorted filter expression for the right child:
+        let right_filter_expression = build_filter_input_order(
+            JoinSide::Right,
+            &filter,
+            &right.schema(),
+            right_ordering,
+        )?;
+
+        // Store the left and right sorted filter expressions in a vector
+        let mut sorted_filter_exprs =
+            vec![left_filter_expression, right_filter_expression];
+
+        // Gather node indices of converted filter expressions in `SortedFilterExpr`
+        // using the filter columns vector:
+        let child_node_indexes = physical_expr_graph.gather_node_indices(
+            &sorted_filter_exprs
+                .iter()
+                .map(|sorted_expr| sorted_expr.filter_expr().clone())
+                .collect::<Vec<_>>(),
+        );
+
+        // Inject calculated node indices into SortedFilterExpr:
+        for (sorted_expr, (_, index)) in sorted_filter_exprs
+            .iter_mut()
+            .zip(child_node_indexes.iter())
+        {
+            sorted_expr.set_node_index(*index);
+        }
+
+        let left_required_sort_exprs = vec![left_ordering.clone()];
+        let right_required_sort_exprs = vec![right_ordering.clone()];
+
+        Ok(SymmetricHashJoinExec {
+            left,
+            right,
+            on,
+            filter,
+            join_type: *join_type,
+            sorted_filter_exprs,
+            left_required_sort_exprs,
+            right_required_sort_exprs,
+            physical_expr_graph,
+            schema: Arc::new(schema),
+            random_state,
+            metrics: ExecutionPlanMetricsSet::new(),
+            column_indices,
+            null_equals_null: *null_equals_null,
+        })
+    }
+
+    /// left stream
+    pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.left
+    }
+
+    /// right stream
+    pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.right
+    }
+
+    /// Set of common columns used to join on
+    pub fn on(&self) -> &[(Column, Column)] {
+        &self.on
+    }
+
+    /// Filters applied before join output
+    pub fn filter(&self) -> &JoinFilter {
+        &self.filter
+    }
+
+    /// How the join is performed
+    pub fn join_type(&self) -> &JoinType {
+        &self.join_type
+    }
+
+    /// Get null_equals_null
+    pub fn null_equals_null(&self) -> &bool {
+        &self.null_equals_null
+    }
+}
+
+impl Debug for SymmetricHashJoinExec {
+    fn fmt(&self, _f: &mut Formatter<'_>) -> fmt::Result {
+        todo!()
+    }
+}
+
+impl ExecutionPlan for SymmetricHashJoinExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
+        vec![
+            Some(&self.left_required_sort_exprs),
+            Some(&self.right_required_sort_exprs),
+        ]
+    }
+
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children.iter().any(|u| *u))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        let (left_expr, right_expr) = self
+            .on
+            .iter()
+            .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
+            .unzip();
+        // TODO: This will change when we extend collected executions.
+        vec![
+            if self.left.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(left_expr)
+            },
+            if self.right.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(right_expr)
+            },
+        ]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        let left_columns_len = self.left.schema().fields.len();
+        partitioned_join_output_partitioning(
+            self.join_type,
+            self.left.output_partitioning(),
+            self.right.output_partitioning(),
+            left_columns_len,
+        )
+    }
+
+    // TODO: Output ordering might be kept for some cases.
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        let left_columns_len = self.left.schema().fields.len();
+        combine_join_equivalence_properties(
+            self.join_type,
+            self.left.equivalence_properties(),
+            self.right.equivalence_properties(),
+            left_columns_len,
+            self.on(),
+            self.schema(),
+        )
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.left.clone(), self.right.clone()]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(SymmetricHashJoinExec::try_new(
+            children[0].clone(),
+            children[1].clone(),
+            self.on.clone(),
+            self.filter.clone(),
+            &self.join_type,
+            &self.null_equals_null,
+        )?))
+    }
+
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                let display_filter = format!(", filter={:?}", self.filter.expression());
+                write!(
+                    f,
+                    "SymmetricHashJoinExec: join_type={:?}, on={:?}{}",
+                    self.join_type, self.on, display_filter
+                )
+            }
+        }
+    }
+
+    fn metrics(&self) -> Option<MetricsSet> {
+        Some(self.metrics.clone_inner())
+    }
+
+    fn statistics(&self) -> Statistics {
+        // TODO stats: it is not possible in general to know the output size of joins
+        Statistics::default()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
+        let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
+        let left_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Left,
+            self.sorted_filter_exprs[0].clone(),
+            on_left,
+            self.left.schema(),
+        );
+        let right_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Right,
+            self.sorted_filter_exprs[1].clone(),
+            on_right,
+            self.right.schema(),
+        );
+        let left_stream = self.left.execute(partition, context.clone())?;
+        let right_stream = self.right.execute(partition, context)?;
+
+        Ok(Box::pin(SymmetricHashJoinStream {
+            left_stream,
+            right_stream,
+            schema: self.schema(),
+            filter: self.filter.clone(),
+            join_type: self.join_type,
+            random_state: self.random_state.clone(),
+            left: left_side_joiner,
+            right: right_side_joiner,
+            column_indices: self.column_indices.clone(),
+            metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics),
+            physical_expr_graph: self.physical_expr_graph.clone(),
+            null_equals_null: self.null_equals_null,
+            final_result: false,
+            probe_side: JoinSide::Left,
+        }))
+    }
+}
+
+/// A stream that issues [RecordBatch]es as they arrive from the right  of the join.
+struct SymmetricHashJoinStream {
+    /// Left stream
+    left_stream: SendableRecordBatchStream,
+    /// right stream
+    right_stream: SendableRecordBatchStream,
+    /// Input schema
+    schema: Arc<Schema>,
+    /// join filter
+    filter: JoinFilter,
+    /// type of the join
+    join_type: JoinType,
+    // left hash joiner
+    left: OneSideHashJoiner,
+    /// right hash joiner
+    right: OneSideHashJoiner,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    // Range pruner.
+    physical_expr_graph: ExprIntervalGraph,
+    /// Random state used for hashing initialization
+    random_state: RandomState,
+    /// If null_equals_null is true, null == null else null != null
+    null_equals_null: bool,
+    /// Metrics
+    metrics: SymmetricHashJoinMetrics,
+    /// Flag indicating whether there is nothing to process anymore
+    final_result: bool,
+    /// The current probe side. We choose build and probe side according to this attribute.
+    probe_side: JoinSide,
+}
+
+impl RecordBatchStream for SymmetricHashJoinStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+impl Stream for SymmetricHashJoinStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Self::Item>> {
+        self.poll_next_impl(cx)
+    }
+}
+
+fn prune_hash_values(
+    prune_length: usize,
+    hashmap: &mut JoinHashMap,
+    row_hash_values: &mut VecDeque<u64>,
+    offset: u64,
+) -> Result<()> {
+    // Create a (hash)-(row number set) map
+    let mut hash_value_map: HashMap<u64, HashSet<u64>> = HashMap::new();
+    for index in 0..prune_length {
+        let hash_value = row_hash_values.pop_front().unwrap();
+        if let Some(set) = hash_value_map.get_mut(&hash_value) {
+            set.insert(offset + index as u64);
+        } else {
+            let mut set = HashSet::new();
+            set.insert(offset + index as u64);
+            hash_value_map.insert(hash_value, set);
+        }
+    }
+    for (hash_value, index_set) in hash_value_map.iter() {
+        if let Some((_, separation_chain)) = hashmap
+            .0
+            .get_mut(*hash_value, |(hash, _)| hash_value == hash)
+        {
+            separation_chain.retain(|n| !index_set.contains(n));
+            if separation_chain.is_empty() {
+                hashmap
+                    .0
+                    .remove_entry(*hash_value, |(hash, _)| hash_value == hash);
+            }
+        }
+    }
+    Ok(())
+}
+
+/// Calculate the filter expression intervals.
+///
+/// This function updates the `interval` field of each `SortedFilterExpr` based
+/// on the first or the last value of the expression in `build_input_buffer`
+/// and `probe_batch`.
+///
+/// # Arguments
+///
+/// * `build_input_buffer` - The [RecordBatch] on the build side of the join.
+/// * `build_sorted_filter_expr` - Build side [SortedFilterExpr] to update.
+/// * `probe_batch` - The `RecordBatch` on the probe side of the join.
+/// * `probe_sorted_filter_expr` - Probe side `SortedFilterExpr` to update.
+///
+/// ### Note
+/// ```text
+///
+/// Interval arithmetic is used to calculate viable join ranges for build-side
+/// pruning. This is done by first creating an interval for join filter values in
+/// the build side of the join, which spans [-∞, FV] or [FV, ∞] depending on the
+/// ordering (descending/ascending) of the filter expression. Here, FV denotes the
+/// first value on the build side. This range is then compared with the probe side
+/// interval, which either spans [-∞, LV] or [LV, ∞] depending on the ordering
+/// (ascending/descending) of the probe side. Here, LV denotes the last value on
+/// the probe side.
+///
+/// As a concrete example, consider the following query:
+///
+///   SELECT * FROM left_table, right_table
+///   WHERE
+///     left_key = right_key AND
+///     a > b - 3 AND
+///     a < b + 10
+///
+/// where columns "a" and "b" come from tables "left_table" and "right_table",
+/// respectively. When a new `RecordBatch` arrives at the right side, the
+/// condition a > b - 3 will possibly indicate a prunable range for the left
+/// side. Conversely, when a new `RecordBatch` arrives at the left side, the
+/// condition a < b + 10 will possibly indicate prunability for the right side.
+/// Let’s inspect what happens when a new RecordBatch` arrives at the right
+/// side (i.e. when the left side is the build side):
+///
+///         Build      Probe
+///       +-------+  +-------+
+///       | a | z |  | b | y |
+///       |+--|--+|  |+--|--+|
+///       | 1 | 2 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 3 | 1 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 5 | 7 |  | 6 | 1 |
+///       |+--|--+|  |+--|--+|
+///       | 7 | 1 |  | 6 | 3 |
+///       +-------+  +-------+
+///
+/// In this case, the interval representing viable (i.e. joinable) values for
+/// column "a" is [1, ∞], and the interval representing possible future values
+/// for column "b" is [6, ∞]. With these intervals at hand, we next calculate
+/// intervals for the whole filter expression and propagate join constraint by
+/// traversing the expression graph.
+/// ```
+fn calculate_filter_expr_intervals(
+    build_input_buffer: &RecordBatch,
+    build_sorted_filter_expr: &mut SortedFilterExpr,
+    probe_batch: &RecordBatch,
+    probe_sorted_filter_expr: &mut SortedFilterExpr,
+) -> Result<()> {
+    // If either build or probe side has no data, return early:
+    if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
+        return Ok(());
+    }
+    // Evaluate build side filter expression and convert the result to an array
+    let build_array = build_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&build_input_buffer.slice(0, 1))?
+        .into_array(1);
+    // Evaluate probe side filter expression and convert the result to an array
+    let probe_array = probe_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))?
+        .into_array(1);
+
+    // Update intervals for both build and probe side filter expressions
+    for (array, sorted_expr) in vec![
+        (build_array, build_sorted_filter_expr),
+        (probe_array, probe_sorted_filter_expr),
+    ] {
+        // Convert the array to a ScalarValue:
+        let value = ScalarValue::try_from_array(&array, 0)?;
+        // Create a ScalarValue representing positive or negative infinity for the same data type:
+        let infinite = ScalarValue::try_from(value.get_datatype())?;
+        // Update the interval with lower and upper bounds based on the sort option
+        sorted_expr.set_interval(
+            if sorted_expr.origin_sorted_expr().options.descending {
+                Interval {
+                    lower: infinite,
+                    upper: value,
+                }
+            } else {
+                Interval {
+                    lower: value,
+                    upper: infinite,
+                }
+            },
+        );
+    }
+    Ok(())
+}
+
+/// Determine the pruning length for `buffer`.
+///
+/// This function evaluates the build side filter expression, converts the
+/// result into an array and determines the pruning length by performing a
+/// binary search on the array.
+///
+/// # Arguments
+///
+/// * `buffer`: The record batch to be pruned.
+/// * `build_side_filter_expr`: The filter expression on the build side used
+/// to determine the pruning length.
+///
+/// # Returns
+///
+/// A [Result] object that contains the pruning length. The function will return
+/// an error if there is an issue evaluating the build side filter expression.
+fn determine_prune_length(
+    buffer: &RecordBatch,
+    build_side_filter_expr: &SortedFilterExpr,
+) -> Result<usize> {
+    let origin_sorted_expr = build_side_filter_expr.origin_sorted_expr();
+    let interval = build_side_filter_expr.interval();
+    // Evaluate the build side filter expression and convert it into an array
+    let batch_arr = origin_sorted_expr
+        .expr
+        .evaluate(buffer)?
+        .into_array(buffer.num_rows());
+
+    // Get the lower or upper interval based on the sort direction
+    let target = if origin_sorted_expr.options.descending {
+        interval.upper.clone()
+    } else {
+        interval.lower.clone()
+    };
+
+    // Perform binary search on the array to determine the length of the record batch to be pruned
+    bisect::<true>(&[batch_arr], &[target], &[origin_sorted_expr.options])
+}
+
+/// This method determines if the result of the join should be produced in the final step or not.
+///
+/// # Arguments
+///
+/// * `build_side` - Enum indicating the side of the join used as the build side.
+/// * `join_type` - Enum indicating the type of join to be performed.
+///
+/// # Returns
+///
+/// A boolean indicating whether the result of the join should be produced in the final step or not.
+/// The result will be true if the build side is JoinSide::Left and the join type is one of
+/// JoinType::Left, JoinType::LeftAnti, JoinType::Full or JoinType::LeftSemi.
+/// If the build side is JoinSide::Right, the result will be true if the join type
+/// is one of JoinType::Right, JoinType::RightAnti, JoinType::Full, or JoinType::RightSemi.
+fn need_to_produce_result_in_final(build_side: JoinSide, join_type: JoinType) -> bool {
+    if build_side == JoinSide::Left {
+        matches!(
+            join_type,
+            JoinType::Left | JoinType::LeftAnti | JoinType::Full | JoinType::LeftSemi
+        )
+    } else {
+        matches!(
+            join_type,
+            JoinType::Right | JoinType::RightAnti | JoinType::Full | JoinType::RightSemi
+        )
+    }
+}
+
+/// Get the anti join indices from the visited hash set.
+///
+/// This method returns the indices from the original input that were not present in the visited hash set.
+///
+/// # Arguments
+///
+/// * `prune_length` - The length of the pruned record batch.
+/// * `deleted_offset` - The offset to the indices.
+/// * `visited_rows` - The hash set of visited indices.
+///
+/// # Returns
+///
+/// A `PrimitiveArray` of the anti join indices.
+fn get_anti_indices<T: ArrowPrimitiveType>(
+    prune_length: usize,
+    deleted_offset: usize,
+    visited_rows: &HashSet<usize>,
+) -> PrimitiveArray<T>
+where
+    NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
+{
+    let mut bitmap = BooleanBufferBuilder::new(prune_length);
+    bitmap.append_n(prune_length, false);
+    // mark the indices as true if they are present in the visited hash set
+    for v in 0..prune_length {
+        let row = v + deleted_offset;
+        bitmap.set_bit(v, visited_rows.contains(&row));
+    }
+    // get the anti index
+    (0..prune_length)
+        .filter_map(|idx| (!bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
+        .collect()
+}
+
+/// This method creates a boolean buffer from the visited rows hash set
+/// and the indices of the pruned record batch slice.
+///
+/// It gets the indices from the original input that were present in the visited hash set.
+///
+/// # Arguments
+///
+/// * `prune_length` - The length of the pruned record batch.
+/// * `deleted_offset` - The offset to the indices.
+/// * `visited_rows` - The hash set of visited indices.
+///
+/// # Returns
+///
+/// A [PrimitiveArray] of the specified type T, containing the semi indices.
+fn get_semi_indices<T: ArrowPrimitiveType>(
+    prune_length: usize,
+    deleted_offset: usize,
+    visited_rows: &HashSet<usize>,
+) -> PrimitiveArray<T>
+where
+    NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
+{
+    let mut bitmap = BooleanBufferBuilder::new(prune_length);
+    bitmap.append_n(prune_length, false);
+    // mark the indices as true if they are present in the visited hash set
+    (0..prune_length).for_each(|v| {
+        let row = &(v + deleted_offset);
+        bitmap.set_bit(v, visited_rows.contains(row));
+    });
+    // get the semi index
+    (0..prune_length)
+        .filter_map(|idx| (bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
+        .collect::<PrimitiveArray<T>>()
+}
+/// Records the visited indices from the input `PrimitiveArray` of type `T` into the given hash set `visited`.
+/// This function will insert the indices (offset by `offset`) into the `visited` hash set.
+///
+/// # Arguments
+///
+/// * `visited` - A hash set to store the visited indices.
+/// * `offset` - An offset to the indices in the `PrimitiveArray`.
+/// * `indices` - The input `PrimitiveArray` of type `T` which stores the indices to be recorded.
+///
+fn record_visited_indices<T: ArrowPrimitiveType>(
+    visited: &mut HashSet<usize>,
+    offset: usize,
+    indices: &PrimitiveArray<T>,
+) {
+    for i in indices.values() {
+        visited.insert(i.as_usize() + offset);
+    }
+}
+
+/// Calculate indices by join type.
+///
+/// This method returns a tuple of two arrays: build and probe indices.
+/// The length of both arrays will be the same.
+///
+/// # Arguments
+///
+/// * `build_side`: Join side which defines the build side.
+/// * `prune_length`: Length of the prune data.
+/// * `visited_rows`: Hash set of visited rows of the build side.
+/// * `deleted_offset`: Deleted offset of the build side.
+/// * `join_type`: The type of join to be performed.
+///
+/// # Returns
+///
+/// A tuple of two arrays of primitive types representing the build and probe indices.
+///
+fn calculate_indices_by_join_type<L: ArrowPrimitiveType, R: ArrowPrimitiveType>(
+    build_side: JoinSide,
+    prune_length: usize,
+    visited_rows: &HashSet<usize>,
+    deleted_offset: usize,
+    join_type: JoinType,
+) -> Result<(PrimitiveArray<L>, PrimitiveArray<R>)>
+where
+    NativeAdapter<L>: From<<L as ArrowPrimitiveType>::Native>,
+{
+    // Store the result in a tuple
+    let result = match (build_side, join_type) {
+        // In the case of `Left` or `Right` join, or `Full` join, get the anti indices
+        (JoinSide::Left, JoinType::Left | JoinType::LeftAnti)
+        | (JoinSide::Right, JoinType::Right | JoinType::RightAnti)
+        | (_, JoinType::Full) => {
+            let build_unmatched_indices =
+                get_anti_indices(prune_length, deleted_offset, visited_rows);
+            let mut builder =
+                PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
+            builder.append_nulls(build_unmatched_indices.len());
+            let probe_indices = builder.finish();
+            (build_unmatched_indices, probe_indices)
+        }
+        // In the case of `LeftSemi` or `RightSemi` join, get the semi indices
+        (JoinSide::Left, JoinType::LeftSemi) | (JoinSide::Right, JoinType::RightSemi) => {
+            let build_unmatched_indices =
+                get_semi_indices(prune_length, deleted_offset, visited_rows);
+            let mut builder =
+                PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
+            builder.append_nulls(build_unmatched_indices.len());
+            let probe_indices = builder.finish();
+            (build_unmatched_indices, probe_indices)
+        }
+        // The case of other join types is not considered
+        _ => unreachable!(),
+    };
+    Ok(result)
+}
+
+struct OneSideHashJoiner {
+    /// Build side
+    build_side: JoinSide,
+    /// Build side filter sort information
+    sorted_filter_expr: SortedFilterExpr,
+    /// Input record batch buffer
+    input_buffer: RecordBatch,
+    /// Columns from the side
+    on: Vec<Column>,
+    /// Hashmap
+    hashmap: JoinHashMap,
+    /// To optimize hash deleting in case of pruning, we hold them in memory
+    row_hash_values: VecDeque<u64>,
+    /// Reuse the hashes buffer
+    hashes_buffer: Vec<u64>,
+    /// Matched rows
+    visited_rows: HashSet<usize>,
+    /// Offset
+    offset: usize,
+    /// Deleted offset
+    deleted_offset: usize,
+    /// Side is exhausted
+    exhausted: bool,
+}
+
+impl OneSideHashJoiner {
+    pub fn new(
+        build_side: JoinSide,
+        sorted_filter_expr: SortedFilterExpr,
+        on: Vec<Column>,
+        schema: SchemaRef,
+    ) -> Self {
+        Self {
+            build_side,
+            input_buffer: RecordBatch::new_empty(schema),
+            on,
+            hashmap: JoinHashMap(RawTable::with_capacity(10_000)),
+            row_hash_values: VecDeque::new(),
+            hashes_buffer: vec![],
+            sorted_filter_expr,
+            visited_rows: HashSet::new(),
+            offset: 0,
+            deleted_offset: 0,
+            exhausted: false,
+        }
+    }
+
+    /// Updates the internal state of the [OneSideHashJoiner] with the incoming batch.
+    ///
+    /// # Arguments
+    ///
+    /// * `batch` - The incoming [RecordBatch] to be merged with the internal input buffer
+    /// * `random_state` - The random state used to hash values
+    ///
+    /// # Returns
+    ///
+    /// Returns a [Result] encapsulating any intermediate errors.
+    fn update_internal_state(
+        &mut self,
+        batch: &RecordBatch,
+        random_state: &RandomState,
+    ) -> Result<()> {
+        // Merge the incoming batch with the existing input buffer:
+        self.input_buffer = concat_batches(&batch.schema(), [&self.input_buffer, batch])?;
+        // Resize the hashes buffer to the number of rows in the incoming batch:
+        self.hashes_buffer.resize(batch.num_rows(), 0);
+        // Update the hashmap with the join key values and hashes of the incoming batch:
+        update_hash(
+            &self.on,
+            batch,
+            &mut self.hashmap,
+            self.offset,
+            random_state,
+            &mut self.hashes_buffer,
+        )?;
+        // Add the hashes buffer to the hash value deque:
+        self.row_hash_values.extend(self.hashes_buffer.iter());
+        Ok(())
+    }
+
+    /// This method performs a join between the build side input buffer and the probe side batch.
+    ///
+    /// # Arguments
+    ///
+    /// * `schema` - A reference to the schema of the output record batch.
+    /// * `join_type` - The type of join to be performed.
+    /// * `on_probe` - An array of columns on which the join will be performed. The columns are from the probe side of the join.
+    /// * `filter` - An optional filter on the join condition.
+    /// * `probe_batch` - The second record batch to be joined.
+    /// * `probe_visited` - A hash set to store the visited indices from the probe batch.
+    /// * `probe_offset` - The offset of the probe side for visited indices calculations.
+    /// * `column_indices` - An array of columns to be selected for the result of the join.
+    /// * `random_state` - The random state for the join.
+    /// * `null_equals_null` - A boolean indicating whether NULL values should be treated as equal when joining.
+    ///
+    /// # Returns
+    ///
+    /// A [Result] containing an optional record batch if the join type is not one of `LeftAnti`, `RightAnti`, `LeftSemi` or `RightSemi`.
+    /// If the join type is one of the above four, the function will return [None].
+    #[allow(clippy::too_many_arguments)]
+    fn join_with_probe_batch(
+        &mut self,
+        schema: &SchemaRef,
+        join_type: JoinType,
+        on_probe: &[Column],
+        filter: &JoinFilter,
+        probe_batch: &RecordBatch,
+        probe_visited: &mut HashSet<usize>,
+        probe_offset: usize,
+        column_indices: &[ColumnIndex],
+        random_state: &RandomState,
+        null_equals_null: &bool,
+    ) -> Result<Option<RecordBatch>> {
+        if self.input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
+            return Ok(Some(RecordBatch::new_empty(schema.clone())));
+        }
+        let (build_indices, probe_indices) = build_join_indices(
+            probe_batch,
+            &self.hashmap,
+            &self.input_buffer,
+            &self.on,
+            on_probe,
+            Some(filter),
+            random_state,
+            null_equals_null,
+            &mut self.hashes_buffer,
+            Some(self.deleted_offset),
+            self.build_side,
+        )?;
+        if need_to_produce_result_in_final(self.build_side, join_type) {
+            record_visited_indices(
+                &mut self.visited_rows,
+                self.deleted_offset,
+                &build_indices,
+            );
+        }
+        if need_to_produce_result_in_final(self.build_side.negate(), join_type) {
+            record_visited_indices(probe_visited, probe_offset, &probe_indices);
+        }
+        if matches!(
+            join_type,
+            JoinType::LeftAnti
+                | JoinType::RightAnti
+                | JoinType::LeftSemi
+                | JoinType::RightSemi
+        ) {
+            Ok(None)
+        } else {
+            build_batch_from_indices(
+                schema,
+                &self.input_buffer,
+                probe_batch,
+                build_indices,
+                probe_indices,
+                column_indices,
+                self.build_side,
+            )
+            .map(Some)
+        }
+    }
+
+    /// This function produces unmatched record results based on the build side,
+    /// join type and other parameters.
+    ///
+    /// The method uses first `prune_length` rows from the build side input buffer
+    /// to produce results.
+    ///
+    /// # Arguments
+    ///
+    /// * `output_schema` - The schema of the final output record batch.
+    /// * `prune_length` - The length of the determined prune length.
+    /// * `probe_schema` - The schema of the probe [RecordBatch].
+    /// * `join_type` - The type of join to be performed.
+    /// * `column_indices` - Indices of columns that are being joined.
+    ///
+    /// # Returns
+    ///
+    /// * `Option<RecordBatch>` - The final output record batch if required, otherwise [None].
+    fn build_side_determined_results(
+        &self,
+        output_schema: &SchemaRef,
+        prune_length: usize,
+        probe_schema: SchemaRef,
+        join_type: JoinType,
+        column_indices: &[ColumnIndex],
+    ) -> Result<Option<RecordBatch>> {
+        // Check if we need to produce a result in the final output:
+        if need_to_produce_result_in_final(self.build_side, join_type) {
+            // Calculate the indices for build and probe sides based on join type and build side:
+            let (build_indices, probe_indices) = calculate_indices_by_join_type(
+                self.build_side,
+                prune_length,
+                &self.visited_rows,
+                self.deleted_offset,
+                join_type,
+            )?;
+
+            // Create an empty probe record batch:
+            let empty_probe_batch = RecordBatch::new_empty(probe_schema);
+            // Build the final result from the indices of build and probe sides:
+            build_batch_from_indices(
+                output_schema.as_ref(),
+                &self.input_buffer,
+                &empty_probe_batch,
+                build_indices,
+                probe_indices,
+                column_indices,
+                self.build_side,
+            )
+            .map(Some)
+        } else {
+            // If we don't need to produce a result, return None
+            Ok(None)
+        }
+    }
+
+    /// Prunes the internal buffer.
+    ///
+    /// Argument `probe_batch` is used to update the intervals of the sorted
+    /// filter expressions. The updated build interval determines the new length
+    /// of the build side. If there are rows to prune, they are removed from the
+    /// internal buffer.
+    ///
+    /// # Arguments
+    ///
+    /// * `schema` - The schema of the final output record batch
+    /// * `probe_batch` - Incoming RecordBatch of the probe side.
+    /// * `probe_side_sorted_filter_expr` - Probe side mutable sorted filter expression.
+    /// * `join_type` - The type of join (e.g. inner, left, right, etc.).
+    /// * `column_indices` - A vector of column indices that specifies which columns from the
+    ///     build side should be included in the output.
+    /// * `physical_expr_graph` - A mutable reference to the physical expression graph.
+    ///
+    /// # Returns
+    ///
+    /// If there are rows to prune, returns the pruned build side record batch wrapped in an `Ok` variant.
+    /// Otherwise, returns `Ok(None)`.
+    fn prune_with_probe_batch(
+        &mut self,
+        schema: &SchemaRef,
+        probe_batch: &RecordBatch,
+        probe_side_sorted_filter_expr: &mut SortedFilterExpr,
+        join_type: JoinType,
+        column_indices: &[ColumnIndex],
+        physical_expr_graph: &mut ExprIntervalGraph,
+    ) -> Result<Option<RecordBatch>> {
+        // Check if the input buffer is empty:
+        if self.input_buffer.num_rows() == 0 {
+            return Ok(None);
+        }
+        // Convert the sorted filter expressions into a vector of (node_index, interval)
+        // tuples for use when updating the interval graph.
+        let mut filter_intervals = vec![
+            (
+                self.sorted_filter_expr.node_index(),
+                self.sorted_filter_expr.interval().clone(),
+            ),
+            (
+                probe_side_sorted_filter_expr.node_index(),
+                probe_side_sorted_filter_expr.interval().clone(),
+            ),
+        ];
+        // Use the join filter intervals to update the physical expression graph:
+        physical_expr_graph.update_ranges(&mut filter_intervals)?;
+        // Get the new join filter interval for build side:
+        let calculated_build_side_interval = filter_intervals.remove(0).1;
+        // Check if the intervals changed, exit early if not:
+        if calculated_build_side_interval.eq(self.sorted_filter_expr.interval()) {
+            return Ok(None);
+        }
+        // Determine the pruning length if there was a change in the intervals:
+        self.sorted_filter_expr
+            .set_interval(calculated_build_side_interval);
+        let prune_length =
+            determine_prune_length(&self.input_buffer, &self.sorted_filter_expr)?;
+        // If we can not prune, exit early:
+        if prune_length == 0 {
+            return Ok(None);
+        }
+        // Compute the result, and perform pruning if there are rows to prune:
+        let result = self.build_side_determined_results(
+            schema,
+            prune_length,
+            probe_batch.schema(),
+            join_type,
+            column_indices,
+        );
+        prune_hash_values(
+            prune_length,
+            &mut self.hashmap,
+            &mut self.row_hash_values,
+            self.deleted_offset as u64,
+        )?;
+        for row in self.deleted_offset..(self.deleted_offset + prune_length) {
+            self.visited_rows.remove(&row);
+        }
+        self.input_buffer = self
+            .input_buffer
+            .slice(prune_length, self.input_buffer.num_rows() - prune_length);
+        self.deleted_offset += prune_length;
+        result
+    }
+}
+
+fn combine_two_batches(
+    output_schema: &SchemaRef,
+    left_batch: Option<RecordBatch>,
+    right_batch: Option<RecordBatch>,
+) -> Result<Option<RecordBatch>> {
+    match (left_batch, right_batch) {
+        (Some(batch), None) | (None, Some(batch)) => {
+            // If only one of the batches are present, return it:
+            Ok(Some(batch))
+        }
+        (Some(left_batch), Some(right_batch)) => {
+            // If both batches are present, concatenate them:
+            concat_batches(output_schema, &[left_batch, right_batch])
+                .map_err(DataFusionError::ArrowError)
+                .map(Some)
+        }
+        (None, None) => {
+            // If neither is present, return an empty batch:
+            Ok(None)
+        }
+    }
+}
+
+impl SymmetricHashJoinStream {
+    /// Polls the next result of the join operation.
+    ///
+    /// If the result of the join is ready, it returns the next record batch.
+    /// If the join has completed and there are no more results, it returns
+    /// `Poll::Ready(None)`. If the join operation is not complete, but the
+    /// current stream is not ready yet, it returns `Poll::Pending`.
+    fn poll_next_impl(
+        &mut self,
+        cx: &mut std::task::Context<'_>,
+    ) -> Poll<Option<Result<RecordBatch>>> {
+        loop {
+            // If the final result has already been obtained, return `Poll::Ready(None)`:
+            if self.final_result {
+                return Poll::Ready(None);
+            }
+            // If both streams have been exhausted, return the final result:
+            if self.right.exhausted && self.left.exhausted {
+                // Get left side results:
+                let left_result = self.left.build_side_determined_results(
+                    &self.schema,
+                    self.left.input_buffer.num_rows(),
+                    self.right.input_buffer.schema(),
+                    self.join_type,
+                    &self.column_indices,
+                )?;
+                // Get right side results:
+                let right_result = self.right.build_side_determined_results(
+                    &self.schema,
+                    self.right.input_buffer.num_rows(),
+                    self.left.input_buffer.schema(),
+                    self.join_type,
+                    &self.column_indices,
+                )?;
+                self.final_result = true;
+                // Combine results:
+                let result =
+                    combine_two_batches(&self.schema, left_result, right_result)?;
+                // Update the metrics if we have a batch; otherwise, continue the loop.
+                if let Some(batch) = &result {
+                    self.metrics.output_batches.add(1);
+                    self.metrics.output_rows.add(batch.num_rows());
+                    return Poll::Ready(Ok(result).transpose());
+                } else {
+                    continue;
+                }
+            }
+
+            // Determine which stream should be polled next. The side the
+            // RecordBatch comes from becomes the probe side.
+            let (
+                input_stream,
+                probe_hash_joiner,
+                build_hash_joiner,
+                build_join_side,
+                probe_side_metrics,
+            ) = if self.probe_side.eq(&JoinSide::Left) {
+                (
+                    &mut self.left_stream,
+                    &mut self.left,
+                    &mut self.right,
+                    JoinSide::Right,
+                    &mut self.metrics.left,
+                )
+            } else {
+                (
+                    &mut self.right_stream,
+                    &mut self.right,
+                    &mut self.left,
+                    JoinSide::Left,
+                    &mut self.metrics.right,
+                )
+            };
+            // Poll the next batch from `input_stream`:
+            match input_stream.poll_next_unpin(cx) {
+                // Batch is available
+                Poll::Ready(Some(Ok(probe_batch))) => {
+                    // Update the metrics for the stream that was polled:
+                    probe_side_metrics.input_batches.add(1);
+                    probe_side_metrics.input_rows.add(probe_batch.num_rows());
+                    // Update the internal state of the hash joiner for the build side:
+                    probe_hash_joiner
+                        .update_internal_state(&probe_batch, &self.random_state)?;
+                    // Calculate filter intervals:
+                    calculate_filter_expr_intervals(
+                        &build_hash_joiner.input_buffer,
+                        &mut build_hash_joiner.sorted_filter_expr,
+                        &probe_batch,
+                        &mut probe_hash_joiner.sorted_filter_expr,
+                    )?;
+                    // Join the two sides:
+                    let equal_result = build_hash_joiner.join_with_probe_batch(
+                        &self.schema,
+                        self.join_type,
+                        &probe_hash_joiner.on,
+                        &self.filter,
+                        &probe_batch,
+                        &mut probe_hash_joiner.visited_rows,
+                        probe_hash_joiner.offset,
+                        &self.column_indices,
+                        &self.random_state,
+                        &self.null_equals_null,
+                    )?;
+                    // Increment the offset for the probe hash joiner:
+                    probe_hash_joiner.offset += probe_batch.num_rows();
+                    // Prune the build side input buffer using the expression
+                    // DAG and filter intervals:
+                    let anti_result = build_hash_joiner.prune_with_probe_batch(
+                        &self.schema,
+                        &probe_batch,
+                        &mut probe_hash_joiner.sorted_filter_expr,
+                        self.join_type,
+                        &self.column_indices,
+                        &mut self.physical_expr_graph,
+                    )?;
+                    // Combine results:
+                    let result =
+                        combine_two_batches(&self.schema, equal_result, anti_result)?;
+                    // Choose next poll side. If the other side is not exhausted,
+                    // switch the probe side before returning the result.
+                    if !build_hash_joiner.exhausted {
+                        self.probe_side = build_join_side;
+                    }
+                    // Update the metrics if we have a batch; otherwise, continue the loop.
+                    if let Some(batch) = &result {
+                        self.metrics.output_batches.add(1);
+                        self.metrics.output_rows.add(batch.num_rows());
+                        return Poll::Ready(Ok(result).transpose());
+                    }
+                }
+                Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))),
+                Poll::Ready(None) => {
+                    // Mark the probe side exhausted:
+                    probe_hash_joiner.exhausted = true;
+                    // Change the probe side:
+                    self.probe_side = build_join_side;
+                }
+                Poll::Pending => {
+                    if !build_hash_joiner.exhausted {
+                        self.probe_side = build_join_side;
+                    } else {
+                        return Poll::Pending;
+                    }
+                }
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use std::fs::File;
+
+    use arrow::array::{Array, ArrayRef};
+    use arrow::array::{Int32Array, TimestampNanosecondArray};
+    use arrow::compute::SortOptions;
+    use arrow::datatypes::{DataType, Field, Schema};
+    use arrow::util::pretty::pretty_format_batches;
+    use rstest::*;
+    use tempfile::TempDir;
+
+    use datafusion_expr::Operator;
+    use datafusion_physical_expr::expressions::{BinaryExpr, Column};
+    use datafusion_physical_expr::intervals::test_utils::gen_conjunctive_numeric_expr;
+    use datafusion_physical_expr::PhysicalExpr;
+
+    use crate::physical_plan::joins::{
+        hash_join_utils::tests::complicated_filter, HashJoinExec, PartitionMode,
+    };
+    use crate::physical_plan::{
+        collect, common, memory::MemoryExec, repartition::RepartitionExec,
+    };
+    use crate::prelude::{SessionConfig, SessionContext};
+    use crate::test_util;
+
+    use super::*;
+
+    const TABLE_SIZE: i32 = 1_000;
+
+    fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) {
+        // compare
+        let first_formatted = pretty_format_batches(collected_1).unwrap().to_string();
+        let second_formatted = pretty_format_batches(collected_2).unwrap().to_string();
+
+        let mut first_formatted_sorted: Vec<&str> =
+            first_formatted.trim().lines().collect();
+        first_formatted_sorted.sort_unstable();
+
+        let mut second_formatted_sorted: Vec<&str> =
+            second_formatted.trim().lines().collect();
+        second_formatted_sorted.sort_unstable();
+
+        for (i, (first_line, second_line)) in first_formatted_sorted
+            .iter()
+            .zip(&second_formatted_sorted)
+            .enumerate()
+        {
+            assert_eq!((i, first_line), (i, second_line));
+        }
+    }
+    #[allow(clippy::too_many_arguments)]
+    async fn partitioned_sym_join_with_filter(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: bool,
+        context: Arc<TaskContext>,
+    ) -> Result<Vec<RecordBatch>> {
+        let partition_count = 4;
+
+        let left_expr = on
+            .iter()
+            .map(|(l, _)| Arc::new(l.clone()) as _)
+            .collect::<Vec<_>>();
+
+        let right_expr = on
+            .iter()
+            .map(|(_, r)| Arc::new(r.clone()) as _)
+            .collect::<Vec<_>>();
+
+        let join = SymmetricHashJoinExec::try_new(
+            Arc::new(RepartitionExec::try_new(
+                left,
+                Partitioning::Hash(left_expr, partition_count),
+            )?),
+            Arc::new(RepartitionExec::try_new(
+                right,
+                Partitioning::Hash(right_expr, partition_count),
+            )?),
+            on,
+            filter,
+            join_type,
+            &null_equals_null,
+        )?;
+
+        let mut batches = vec![];
+        for i in 0..partition_count {
+            let stream = join.execute(i, context.clone())?;
+            let more_batches = common::collect(stream).await?;
+            batches.extend(
+                more_batches
+                    .into_iter()
+                    .filter(|b| b.num_rows() > 0)
+                    .collect::<Vec<_>>(),
+            );
+        }
+
+        Ok(batches)
+    }
+    #[allow(clippy::too_many_arguments)]
+    async fn partitioned_hash_join_with_filter(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: bool,
+        context: Arc<TaskContext>,
+    ) -> Result<Vec<RecordBatch>> {
+        let partition_count = 4;
+
+        let (left_expr, right_expr) = on
+            .iter()
+            .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
+            .unzip();
+
+        let join = HashJoinExec::try_new(
+            Arc::new(RepartitionExec::try_new(
+                left,
+                Partitioning::Hash(left_expr, partition_count),
+            )?),
+            Arc::new(RepartitionExec::try_new(
+                right,
+                Partitioning::Hash(right_expr, partition_count),
+            )?),
+            on,
+            Some(filter),
+            join_type,
+            PartitionMode::Partitioned,
+            &null_equals_null,
+        )?;
+
+        let mut batches = vec![];
+        for i in 0..partition_count {
+            let stream = join.execute(i, context.clone())?;
+            let more_batches = common::collect(stream).await?;
+            batches.extend(
+                more_batches
+                    .into_iter()
+                    .filter(|b| b.num_rows() > 0)
+                    .collect::<Vec<_>>(),
+            );
+        }
+
+        Ok(batches)
+    }
+
+    pub fn split_record_batches(
+        batch: &RecordBatch,
+        batch_size: usize,
+    ) -> Result<Vec<RecordBatch>> {
+        let row_num = batch.num_rows();
+        let number_of_batch = row_num / batch_size;
+        let mut sizes = vec![batch_size; number_of_batch];
+        sizes.push(row_num - (batch_size * number_of_batch));
+        let mut result = vec![];
+        for (i, size) in sizes.iter().enumerate() {
+            result.push(batch.slice(i * batch_size, *size));
+        }
+        Ok(result)
+    }
+
+    fn build_record_batch(columns: Vec<(&str, ArrayRef)>) -> Result<RecordBatch> {

Review Comment:
   This looks very similar to https://docs.rs/arrow/33.0.0/arrow/record_batch/struct.RecordBatch.html#method.try_from_iter



##########
datafusion/core/src/physical_plan/memory.rs:
##########
@@ -45,6 +45,8 @@ pub struct MemoryExec {
     projected_schema: SchemaRef,
     /// Optional projection
     projection: Option<Vec<usize>>,
+    // Optional sort information

Review Comment:
   πŸ‘  that is a nice addition



##########
datafusion/physical-expr/src/intervals/interval_aritmetic.rs:
##########
@@ -0,0 +1,533 @@
+// 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.
+
+//! Interval arithmetic library
+
+use std::borrow::Borrow;
+use std::fmt;
+use std::fmt::{Display, Formatter};
+
+use arrow::compute::{cast_with_options, CastOptions};
+use arrow::datatypes::DataType;
+use datafusion_common::{DataFusionError, Result, ScalarValue};
+use datafusion_expr::Operator;
+
+use crate::aggregate::min_max::{max, min};
+
+/// This type represents an interval, which is used to calculate reliable
+/// bounds for expressions. Currently, we only support addition and
+/// subtraction, but more capabilities will be added in the future.
+/// Upper/lower bounds having NULL values indicate an unbounded side. For
+/// example; [10, 20], [10, ∞], [-∞, 100] and [-∞, ∞] are all valid intervals.
+#[derive(Debug, PartialEq, Clone, Eq, Hash)]
+pub struct Interval {
+    pub lower: ScalarValue,
+    pub upper: ScalarValue,
+}
+
+impl Default for Interval {
+    fn default() -> Self {
+        Interval {
+            lower: ScalarValue::Null,
+            upper: ScalarValue::Null,
+        }
+    }
+}
+
+impl Display for Interval {
+    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
+        write!(f, "Interval [{}, {}]", self.lower, self.upper)
+    }
+}
+
+impl Interval {
+    pub(crate) fn cast_to(
+        &self,
+        data_type: &DataType,
+        cast_options: &CastOptions,
+    ) -> Result<Interval> {
+        Ok(Interval {
+            lower: cast_scalar_value(&self.lower, data_type, cast_options)?,
+            upper: cast_scalar_value(&self.upper, data_type, cast_options)?,
+        })
+    }
+
+    pub(crate) fn get_datatype(&self) -> DataType {
+        self.lower.get_datatype()
+    }
+
+    /// Decide if this interval is certainly greater than, possibly greater than,
+    /// or can't be greater than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn gt(&self, other: &Interval) -> Interval {

Review Comment:
   I understand the rationale for putting the analysis logic into `Interval` itself (to keep interval analysis self contained). 
   
   However, it seems to be another possibility would be to move this logic into the `PhysicalExpr` implementation of `BinaryExpr`. The benefit of keeping the interval logic in the `PhysicalExpr` is 
   1. that there is not an implicit coupling of the implementation of the physical exprs and the interval calculations that must be kept in sync. 
   2. When people extend DataFusion with their own `PhysicalExpr`s . the range analysis can support those operators too (rather than just the built in operators)
   
   The second reasons is far more important I think.
   
   I wonder if we could remove the `Interval::gt` `Interval::lt`, etc funtions and instead use the existing range analysis in the operators such as
   
   https://github.com/apache/arrow-datafusion/blob/095f9348455880e6169c09a7c02edfe231aac59a/datafusion/physical-expr/src/expressions/binary.rs#L759-L786
   
   (and maybe change them to be in terms of Intervals?)
   
   



##########
datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs:
##########
@@ -0,0 +1,2640 @@
+// 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 the symmetric hash join algorithm with range-based
+//! data pruning to join two (potentially infinite) streams.
+//!
+//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate
+//! output ordering) and produces the join output according to the given join
+//! type and other options.
+//!
+//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations
+//! for both its children.
+
+use std::collections::{HashMap, VecDeque};
+use std::fmt;
+use std::fmt::{Debug, Formatter};
+use std::sync::Arc;
+use std::task::Poll;
+use std::vec;
+use std::{any::Any, usize};
+
+use ahash::RandomState;
+use arrow::array::{
+    ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray,
+    PrimitiveBuilder,
+};
+use arrow::compute::concat_batches;
+use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef};
+use arrow::record_batch::RecordBatch;
+use futures::{Stream, StreamExt};
+use hashbrown::{raw::RawTable, HashSet};
+
+use datafusion_common::{utils::bisect, ScalarValue};
+use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval};
+
+use crate::error::{DataFusionError, Result};
+use crate::execution::context::TaskContext;
+use crate::logical_expr::JoinType;
+use crate::physical_plan::{
+    expressions::Column,
+    expressions::PhysicalSortExpr,
+    joins::{
+        hash_join::{build_join_indices, update_hash, JoinHashMap},
+        hash_join_utils::{build_filter_input_order, SortedFilterExpr},
+        utils::{
+            build_batch_from_indices, build_join_schema, check_join_is_valid,
+            combine_join_equivalence_properties, partitioned_join_output_partitioning,
+            ColumnIndex, JoinFilter, JoinOn, JoinSide,
+        },
+    },
+    metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet},
+    DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning,
+    RecordBatchStream, SendableRecordBatchStream, Statistics,
+};
+
+/// A symmetric hash join with range conditions is when both streams are hashed on the
+/// join key and the resulting hash tables are used to join the streams.
+/// The join is considered symmetric because the hash table is built on the join keys from both
+/// streams, and the matching of rows is based on the values of the join keys in both streams.
+/// This type of join is efficient in streaming context as it allows for fast lookups in the hash
+/// table, rather than having to scan through one or both of the streams to find matching rows, also it
+/// only considers the elements from the stream that fall within a certain sliding window (w/ range conditions),
+/// making it more efficient and less likely to store stale data. This enables operating on unbounded streaming
+/// data without any memory issues.
+///
+/// For each input stream, create a hash table.
+///   - For each new [RecordBatch] in build side, hash and insert into inputs hash table. Update offsets.
+///   - Test if input is equal to a predefined set of other inputs.
+///   - If so record the visited rows. If the matched row results must be produced (INNER, LEFT), output the [RecordBatch].
+///   - Try to prune other side (probe) with new [RecordBatch].
+///   - If the join type indicates that the unmatched rows results must be produced (LEFT, FULL etc.),
+/// output the [RecordBatch] when a pruning happens or at the end of the data.
+///
+///
+/// ``` text
+///                        +-------------------------+
+///                        |                         |
+///   left stream ---------|  Left OneSideHashJoiner |---+
+///                        |                         |   |
+///                        +-------------------------+   |
+///                                                      |
+///                                                      |--------- Joined output
+///                                                      |
+///                        +-------------------------+   |
+///                        |                         |   |
+///  right stream ---------| Right OneSideHashJoiner |---+
+///                        |                         |
+///                        +-------------------------+
+///
+/// Prune build side when the new RecordBatch comes to the probe side. We utilize interval arithmetic
+/// on JoinFilter's sorted PhysicalExprs to calculate the joinable range.
+///
+///
+///               PROBE SIDE          BUILD SIDE
+///                 BUFFER              BUFFER
+///             +-------------+     +------------+
+///             |             |     |            |    Unjoinable
+///             |             |     |            |    Range
+///             |             |     |            |
+///             |             |  |---------------------------------
+///             |             |  |  |            |
+///             |             |  |  |            |
+///             |             | /   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |
+///             |             | |   |            |    Joinable
+///             |             |/    |            |    Range
+///             |             ||    |            |
+///             |+-----------+||    |            |
+///             || Record    ||     |            |
+///             || Batch     ||     |            |
+///             |+-----------+||    |            |
+///             +-------------+\    +------------+
+///                             |
+///                             \
+///                              |---------------------------------
+///
+///  This happens when range conditions are provided on sorted columns. E.g.
+///
+///        SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_time > right_time - INTERVAL 12 MINUTES AND left_time < right_time + INTERVAL 2 HOUR
+///
+/// or
+///       SELECT * FROM left_table, right_table
+///        ON
+///          left_key = right_key AND
+///          left_sorted > right_sorted - 3 AND left_sorted < right_sorted + 10
+///
+/// For general purpose, in the second scenario, when the new data comes to probe side, the conditions can be used to
+/// determine a specific threshold for discarding rows from the inner buffer. For example, if the sort order the
+/// two columns ("left_sorted" and "right_sorted") are ascending (it can be different in another scenarios)
+/// and the join condition is "left_sorted > right_sorted - 3" and the latest value on the right input is 1234, meaning
+/// that the left side buffer must only keep rows where "leftTime > rightTime - 3 > 1234 - 3 > 1231" ,
+/// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending)
+/// than that can be dropped from the inner buffer.
+/// ```
+pub struct SymmetricHashJoinExec {
+    /// Left side stream
+    pub(crate) left: Arc<dyn ExecutionPlan>,
+    /// Right side stream
+    pub(crate) right: Arc<dyn ExecutionPlan>,
+    /// Set of common columns used to join on
+    pub(crate) on: Vec<(Column, Column)>,
+    /// Filters applied when finding matching rows
+    pub(crate) filter: JoinFilter,
+    /// How the join is performed
+    pub(crate) join_type: JoinType,
+    /// Order information of filter expressions
+    sorted_filter_exprs: Vec<SortedFilterExpr>,
+    /// Left required sort
+    left_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Right required sort
+    right_required_sort_exprs: Vec<PhysicalSortExpr>,
+    /// Expression graph for interval calculations
+    physical_expr_graph: ExprIntervalGraph,
+    /// The schema once the join is applied
+    schema: SchemaRef,
+    /// Shares the `RandomState` for the hashing algorithm
+    random_state: RandomState,
+    /// Execution metrics
+    metrics: ExecutionPlanMetricsSet,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    /// If null_equals_null is true, null == null else null != null
+    pub(crate) null_equals_null: bool,
+}
+
+#[derive(Debug)]
+struct SymmetricHashJoinSideMetrics {
+    /// Number of batches consumed by this operator
+    input_batches: metrics::Count,
+    /// Number of rows consumed by this operator
+    input_rows: metrics::Count,
+}
+
+/// Metrics for HashJoinExec
+#[derive(Debug)]
+struct SymmetricHashJoinMetrics {
+    /// Number of left batches/rows consumed by this operator
+    left: SymmetricHashJoinSideMetrics,
+    /// Number of right batches/rows consumed by this operator
+    right: SymmetricHashJoinSideMetrics,
+    /// Number of batches produced by this operator
+    output_batches: metrics::Count,
+    /// Number of rows produced by this operator
+    output_rows: metrics::Count,
+}
+
+impl SymmetricHashJoinMetrics {
+    pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self {
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let left = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let input_batches =
+            MetricBuilder::new(metrics).counter("input_batches", partition);
+        let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
+        let right = SymmetricHashJoinSideMetrics {
+            input_batches,
+            input_rows,
+        };
+
+        let output_batches =
+            MetricBuilder::new(metrics).counter("output_batches", partition);
+
+        let output_rows = MetricBuilder::new(metrics).output_rows(partition);
+
+        Self {
+            left,
+            right,
+            output_batches,
+            output_rows,
+        }
+    }
+}
+
+impl SymmetricHashJoinExec {
+    /// Tries to create a new [SymmetricHashJoinExec].
+    /// # Error
+    /// This function errors when:
+    /// - It is not possible to join the left and right sides on keys `on`, or
+    /// - It fails to construct [SortedFilterExpr]s, or
+    /// - It fails to create the [ExprIntervalGraph].
+    pub fn try_new(
+        left: Arc<dyn ExecutionPlan>,
+        right: Arc<dyn ExecutionPlan>,
+        on: JoinOn,
+        filter: JoinFilter,
+        join_type: &JoinType,
+        null_equals_null: &bool,
+    ) -> Result<Self> {
+        let left_schema = left.schema();
+        let right_schema = right.schema();
+
+        // Error out if no "on" contraints are given:
+        if on.is_empty() {
+            return Err(DataFusionError::Plan(
+                "On constraints in SymmetricHashJoinExec should be non-empty".to_string(),
+            ));
+        }
+
+        // Check if the join is valid with the given on constraints:
+        check_join_is_valid(&left_schema, &right_schema, &on)?;
+
+        // Build the join schema from the left and right schemas:
+        let (schema, column_indices) =
+            build_join_schema(&left_schema, &right_schema, join_type);
+
+        // Set a random state for the join:
+        let random_state = RandomState::with_seeds(0, 0, 0, 0);
+
+        // Create an expression DAG for the join filter:
+        let mut physical_expr_graph =
+            ExprIntervalGraph::try_new(filter.expression().clone())?;
+
+        // Interval calculations require each column to exhibit monotonicity
+        // independently. However, a `PhysicalSortExpr` object defines a
+        // lexicographical ordering, so we can only use their first elements.
+        // when deducing column monotonicities.
+        // TODO: Extend the `PhysicalSortExpr` mechanism to express independent
+        //       (i.e. simultaneous) ordering properties of columns.
+        let (left_ordering, right_ordering) = match (
+            left.output_ordering(),
+            right.output_ordering(),
+        ) {
+            (Some([left_ordering, ..]), Some([right_ordering, ..])) => {
+                (left_ordering, right_ordering)
+            }
+            _ => {
+                return Err(DataFusionError::Plan(
+                    "Symmetric hash join requires its children to have an output ordering".to_string(),
+                ));
+            }
+        };
+
+        // Build the sorted filter expression for the left child:
+        let left_filter_expression = build_filter_input_order(
+            JoinSide::Left,
+            &filter,
+            &left.schema(),
+            left_ordering,
+        )?;
+
+        // Build the sorted filter expression for the right child:
+        let right_filter_expression = build_filter_input_order(
+            JoinSide::Right,
+            &filter,
+            &right.schema(),
+            right_ordering,
+        )?;
+
+        // Store the left and right sorted filter expressions in a vector
+        let mut sorted_filter_exprs =
+            vec![left_filter_expression, right_filter_expression];
+
+        // Gather node indices of converted filter expressions in `SortedFilterExpr`
+        // using the filter columns vector:
+        let child_node_indexes = physical_expr_graph.gather_node_indices(
+            &sorted_filter_exprs
+                .iter()
+                .map(|sorted_expr| sorted_expr.filter_expr().clone())
+                .collect::<Vec<_>>(),
+        );
+
+        // Inject calculated node indices into SortedFilterExpr:
+        for (sorted_expr, (_, index)) in sorted_filter_exprs
+            .iter_mut()
+            .zip(child_node_indexes.iter())
+        {
+            sorted_expr.set_node_index(*index);
+        }
+
+        let left_required_sort_exprs = vec![left_ordering.clone()];
+        let right_required_sort_exprs = vec![right_ordering.clone()];
+
+        Ok(SymmetricHashJoinExec {
+            left,
+            right,
+            on,
+            filter,
+            join_type: *join_type,
+            sorted_filter_exprs,
+            left_required_sort_exprs,
+            right_required_sort_exprs,
+            physical_expr_graph,
+            schema: Arc::new(schema),
+            random_state,
+            metrics: ExecutionPlanMetricsSet::new(),
+            column_indices,
+            null_equals_null: *null_equals_null,
+        })
+    }
+
+    /// left stream
+    pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.left
+    }
+
+    /// right stream
+    pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
+        &self.right
+    }
+
+    /// Set of common columns used to join on
+    pub fn on(&self) -> &[(Column, Column)] {
+        &self.on
+    }
+
+    /// Filters applied before join output
+    pub fn filter(&self) -> &JoinFilter {
+        &self.filter
+    }
+
+    /// How the join is performed
+    pub fn join_type(&self) -> &JoinType {
+        &self.join_type
+    }
+
+    /// Get null_equals_null
+    pub fn null_equals_null(&self) -> &bool {
+        &self.null_equals_null
+    }
+}
+
+impl Debug for SymmetricHashJoinExec {
+    fn fmt(&self, _f: &mut Formatter<'_>) -> fmt::Result {
+        todo!()
+    }
+}
+
+impl ExecutionPlan for SymmetricHashJoinExec {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+
+    fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
+        vec![
+            Some(&self.left_required_sort_exprs),
+            Some(&self.right_required_sort_exprs),
+        ]
+    }
+
+    fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
+        Ok(children.iter().any(|u| *u))
+    }
+
+    fn required_input_distribution(&self) -> Vec<Distribution> {
+        let (left_expr, right_expr) = self
+            .on
+            .iter()
+            .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
+            .unzip();
+        // TODO: This will change when we extend collected executions.
+        vec![
+            if self.left.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(left_expr)
+            },
+            if self.right.output_partitioning().partition_count() == 1 {
+                Distribution::SinglePartition
+            } else {
+                Distribution::HashPartitioned(right_expr)
+            },
+        ]
+    }
+
+    fn output_partitioning(&self) -> Partitioning {
+        let left_columns_len = self.left.schema().fields.len();
+        partitioned_join_output_partitioning(
+            self.join_type,
+            self.left.output_partitioning(),
+            self.right.output_partitioning(),
+            left_columns_len,
+        )
+    }
+
+    // TODO: Output ordering might be kept for some cases.
+    fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
+        None
+    }
+
+    fn equivalence_properties(&self) -> EquivalenceProperties {
+        let left_columns_len = self.left.schema().fields.len();
+        combine_join_equivalence_properties(
+            self.join_type,
+            self.left.equivalence_properties(),
+            self.right.equivalence_properties(),
+            left_columns_len,
+            self.on(),
+            self.schema(),
+        )
+    }
+
+    fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
+        vec![self.left.clone(), self.right.clone()]
+    }
+
+    fn with_new_children(
+        self: Arc<Self>,
+        children: Vec<Arc<dyn ExecutionPlan>>,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        Ok(Arc::new(SymmetricHashJoinExec::try_new(
+            children[0].clone(),
+            children[1].clone(),
+            self.on.clone(),
+            self.filter.clone(),
+            &self.join_type,
+            &self.null_equals_null,
+        )?))
+    }
+
+    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
+        match t {
+            DisplayFormatType::Default => {
+                let display_filter = format!(", filter={:?}", self.filter.expression());
+                write!(
+                    f,
+                    "SymmetricHashJoinExec: join_type={:?}, on={:?}{}",
+                    self.join_type, self.on, display_filter
+                )
+            }
+        }
+    }
+
+    fn metrics(&self) -> Option<MetricsSet> {
+        Some(self.metrics.clone_inner())
+    }
+
+    fn statistics(&self) -> Statistics {
+        // TODO stats: it is not possible in general to know the output size of joins
+        Statistics::default()
+    }
+
+    fn execute(
+        &self,
+        partition: usize,
+        context: Arc<TaskContext>,
+    ) -> Result<SendableRecordBatchStream> {
+        let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
+        let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
+        let left_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Left,
+            self.sorted_filter_exprs[0].clone(),
+            on_left,
+            self.left.schema(),
+        );
+        let right_side_joiner = OneSideHashJoiner::new(
+            JoinSide::Right,
+            self.sorted_filter_exprs[1].clone(),
+            on_right,
+            self.right.schema(),
+        );
+        let left_stream = self.left.execute(partition, context.clone())?;
+        let right_stream = self.right.execute(partition, context)?;
+
+        Ok(Box::pin(SymmetricHashJoinStream {
+            left_stream,
+            right_stream,
+            schema: self.schema(),
+            filter: self.filter.clone(),
+            join_type: self.join_type,
+            random_state: self.random_state.clone(),
+            left: left_side_joiner,
+            right: right_side_joiner,
+            column_indices: self.column_indices.clone(),
+            metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics),
+            physical_expr_graph: self.physical_expr_graph.clone(),
+            null_equals_null: self.null_equals_null,
+            final_result: false,
+            probe_side: JoinSide::Left,
+        }))
+    }
+}
+
+/// A stream that issues [RecordBatch]es as they arrive from the right  of the join.
+struct SymmetricHashJoinStream {
+    /// Left stream
+    left_stream: SendableRecordBatchStream,
+    /// right stream
+    right_stream: SendableRecordBatchStream,
+    /// Input schema
+    schema: Arc<Schema>,
+    /// join filter
+    filter: JoinFilter,
+    /// type of the join
+    join_type: JoinType,
+    // left hash joiner
+    left: OneSideHashJoiner,
+    /// right hash joiner
+    right: OneSideHashJoiner,
+    /// Information of index and left / right placement of columns
+    column_indices: Vec<ColumnIndex>,
+    // Range pruner.
+    physical_expr_graph: ExprIntervalGraph,
+    /// Random state used for hashing initialization
+    random_state: RandomState,
+    /// If null_equals_null is true, null == null else null != null
+    null_equals_null: bool,
+    /// Metrics
+    metrics: SymmetricHashJoinMetrics,
+    /// Flag indicating whether there is nothing to process anymore
+    final_result: bool,
+    /// The current probe side. We choose build and probe side according to this attribute.
+    probe_side: JoinSide,
+}
+
+impl RecordBatchStream for SymmetricHashJoinStream {
+    fn schema(&self) -> SchemaRef {
+        self.schema.clone()
+    }
+}
+
+impl Stream for SymmetricHashJoinStream {
+    type Item = Result<RecordBatch>;
+
+    fn poll_next(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Option<Self::Item>> {
+        self.poll_next_impl(cx)
+    }
+}
+
+fn prune_hash_values(
+    prune_length: usize,
+    hashmap: &mut JoinHashMap,
+    row_hash_values: &mut VecDeque<u64>,
+    offset: u64,
+) -> Result<()> {
+    // Create a (hash)-(row number set) map
+    let mut hash_value_map: HashMap<u64, HashSet<u64>> = HashMap::new();
+    for index in 0..prune_length {
+        let hash_value = row_hash_values.pop_front().unwrap();
+        if let Some(set) = hash_value_map.get_mut(&hash_value) {
+            set.insert(offset + index as u64);
+        } else {
+            let mut set = HashSet::new();
+            set.insert(offset + index as u64);
+            hash_value_map.insert(hash_value, set);
+        }
+    }
+    for (hash_value, index_set) in hash_value_map.iter() {
+        if let Some((_, separation_chain)) = hashmap
+            .0
+            .get_mut(*hash_value, |(hash, _)| hash_value == hash)
+        {
+            separation_chain.retain(|n| !index_set.contains(n));
+            if separation_chain.is_empty() {
+                hashmap
+                    .0
+                    .remove_entry(*hash_value, |(hash, _)| hash_value == hash);
+            }
+        }
+    }
+    Ok(())
+}
+
+/// Calculate the filter expression intervals.
+///
+/// This function updates the `interval` field of each `SortedFilterExpr` based
+/// on the first or the last value of the expression in `build_input_buffer`
+/// and `probe_batch`.
+///
+/// # Arguments
+///
+/// * `build_input_buffer` - The [RecordBatch] on the build side of the join.
+/// * `build_sorted_filter_expr` - Build side [SortedFilterExpr] to update.
+/// * `probe_batch` - The `RecordBatch` on the probe side of the join.
+/// * `probe_sorted_filter_expr` - Probe side `SortedFilterExpr` to update.
+///
+/// ### Note
+/// ```text
+///
+/// Interval arithmetic is used to calculate viable join ranges for build-side
+/// pruning. This is done by first creating an interval for join filter values in
+/// the build side of the join, which spans [-∞, FV] or [FV, ∞] depending on the
+/// ordering (descending/ascending) of the filter expression. Here, FV denotes the
+/// first value on the build side. This range is then compared with the probe side
+/// interval, which either spans [-∞, LV] or [LV, ∞] depending on the ordering
+/// (ascending/descending) of the probe side. Here, LV denotes the last value on
+/// the probe side.
+///
+/// As a concrete example, consider the following query:
+///
+///   SELECT * FROM left_table, right_table
+///   WHERE
+///     left_key = right_key AND
+///     a > b - 3 AND
+///     a < b + 10
+///
+/// where columns "a" and "b" come from tables "left_table" and "right_table",
+/// respectively. When a new `RecordBatch` arrives at the right side, the
+/// condition a > b - 3 will possibly indicate a prunable range for the left
+/// side. Conversely, when a new `RecordBatch` arrives at the left side, the
+/// condition a < b + 10 will possibly indicate prunability for the right side.
+/// Let’s inspect what happens when a new RecordBatch` arrives at the right
+/// side (i.e. when the left side is the build side):
+///
+///         Build      Probe
+///       +-------+  +-------+
+///       | a | z |  | b | y |
+///       |+--|--+|  |+--|--+|
+///       | 1 | 2 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 3 | 1 |  | 4 | 3 |
+///       |+--|--+|  |+--|--+|
+///       | 5 | 7 |  | 6 | 1 |
+///       |+--|--+|  |+--|--+|
+///       | 7 | 1 |  | 6 | 3 |
+///       +-------+  +-------+
+///
+/// In this case, the interval representing viable (i.e. joinable) values for
+/// column "a" is [1, ∞], and the interval representing possible future values
+/// for column "b" is [6, ∞]. With these intervals at hand, we next calculate
+/// intervals for the whole filter expression and propagate join constraint by
+/// traversing the expression graph.
+/// ```
+fn calculate_filter_expr_intervals(
+    build_input_buffer: &RecordBatch,
+    build_sorted_filter_expr: &mut SortedFilterExpr,
+    probe_batch: &RecordBatch,
+    probe_sorted_filter_expr: &mut SortedFilterExpr,
+) -> Result<()> {
+    // If either build or probe side has no data, return early:
+    if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
+        return Ok(());
+    }
+    // Evaluate build side filter expression and convert the result to an array
+    let build_array = build_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&build_input_buffer.slice(0, 1))?
+        .into_array(1);
+    // Evaluate probe side filter expression and convert the result to an array
+    let probe_array = probe_sorted_filter_expr
+        .origin_sorted_expr()
+        .expr
+        .evaluate(&probe_batch.slice(probe_batch.num_rows() - 1, 1))?
+        .into_array(1);
+
+    // Update intervals for both build and probe side filter expressions
+    for (array, sorted_expr) in vec![
+        (build_array, build_sorted_filter_expr),
+        (probe_array, probe_sorted_filter_expr),
+    ] {
+        // Convert the array to a ScalarValue:
+        let value = ScalarValue::try_from_array(&array, 0)?;
+        // Create a ScalarValue representing positive or negative infinity for the same data type:

Review Comment:
   πŸ‘  I double checked that null represents invite (it is nicely explained in the comments of `Interval`)



##########
datafusion/physical-expr/src/intervals/interval_aritmetic.rs:
##########
@@ -0,0 +1,533 @@
+// 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.
+
+//! Interval arithmetic library
+
+use std::borrow::Borrow;
+use std::fmt;
+use std::fmt::{Display, Formatter};
+
+use arrow::compute::{cast_with_options, CastOptions};
+use arrow::datatypes::DataType;
+use datafusion_common::{DataFusionError, Result, ScalarValue};
+use datafusion_expr::Operator;
+
+use crate::aggregate::min_max::{max, min};
+
+/// This type represents an interval, which is used to calculate reliable
+/// bounds for expressions. Currently, we only support addition and
+/// subtraction, but more capabilities will be added in the future.
+/// Upper/lower bounds having NULL values indicate an unbounded side. For
+/// example; [10, 20], [10, ∞], [-∞, 100] and [-∞, ∞] are all valid intervals.
+#[derive(Debug, PartialEq, Clone, Eq, Hash)]
+pub struct Interval {
+    pub lower: ScalarValue,
+    pub upper: ScalarValue,
+}
+
+impl Default for Interval {
+    fn default() -> Self {
+        Interval {
+            lower: ScalarValue::Null,
+            upper: ScalarValue::Null,
+        }
+    }
+}
+
+impl Display for Interval {
+    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
+        write!(f, "Interval [{}, {}]", self.lower, self.upper)
+    }
+}
+
+impl Interval {
+    pub(crate) fn cast_to(
+        &self,
+        data_type: &DataType,
+        cast_options: &CastOptions,
+    ) -> Result<Interval> {
+        Ok(Interval {
+            lower: cast_scalar_value(&self.lower, data_type, cast_options)?,
+            upper: cast_scalar_value(&self.upper, data_type, cast_options)?,
+        })
+    }
+
+    pub(crate) fn get_datatype(&self) -> DataType {
+        self.lower.get_datatype()
+    }
+
+    /// Decide if this interval is certainly greater than, possibly greater than,
+    /// or can't be greater than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn gt(&self, other: &Interval) -> Interval {
+        let flags = if !self.upper.is_null()
+            && !other.lower.is_null()
+            && (self.upper <= other.lower)
+        {
+            (false, false)
+        } else if !self.lower.is_null()
+            && !other.upper.is_null()
+            && (self.lower > other.upper)
+        {
+            (true, true)
+        } else {
+            (false, true)
+        };
+        Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        }
+    }
+
+    /// Decide if this interval is certainly less than, possibly less than,
+    /// or can't be less than `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.
+    pub(crate) fn lt(&self, other: &Interval) -> Interval {
+        other.gt(self)
+    }
+
+    /// Decide if this interval is certainly equal to, possibly equal to,
+    /// or can't be equal to `other` by returning [true, true],
+    /// [false, true] or [false, false] respectively.    
+    pub(crate) fn equal(&self, other: &Interval) -> Interval {
+        let flags = if !self.lower.is_null()
+            && (self.lower == self.upper)
+            && (other.lower == other.upper)
+            && (self.lower == other.lower)
+        {
+            (true, true)
+        } else if (!self.lower.is_null()
+            && !other.upper.is_null()
+            && (self.lower > other.upper))
+            || (!self.upper.is_null()
+                && !other.lower.is_null()
+                && (self.upper < other.lower))
+        {
+            (false, false)
+        } else {
+            (false, true)
+        };
+        Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        }
+    }
+
+    /// Compute the logical conjunction of this (boolean) interval with the
+    /// given boolean interval.
+    pub(crate) fn and(&self, other: &Interval) -> Result<Interval> {
+        let flags = match (self, other) {
+            (
+                Interval {
+                    lower: ScalarValue::Boolean(Some(lower)),
+                    upper: ScalarValue::Boolean(Some(upper)),
+                },
+                Interval {
+                    lower: ScalarValue::Boolean(Some(other_lower)),
+                    upper: ScalarValue::Boolean(Some(other_upper)),
+                },
+            ) => {
+                if *lower && *other_lower {
+                    (true, true)
+                } else if *upper && *other_upper {
+                    (false, true)
+                } else {
+                    (false, false)
+                }
+            }
+            _ => {
+                return Err(DataFusionError::Internal(
+                    "Incompatible types for logical conjunction".to_string(),
+                ))
+            }
+        };
+        Ok(Interval {
+            lower: ScalarValue::Boolean(Some(flags.0)),
+            upper: ScalarValue::Boolean(Some(flags.1)),
+        })
+    }
+
+    /// Compute the intersection of the interval with the given interval.
+    /// If the intersection is empty, return None.
+    pub(crate) fn intersect(&self, other: &Interval) -> Result<Option<Interval>> {
+        let lower = if self.lower.is_null() {
+            other.lower.clone()
+        } else if other.lower.is_null() {
+            self.lower.clone()
+        } else {
+            max(&self.lower, &other.lower)?
+        };
+        let upper = if self.upper.is_null() {
+            other.upper.clone()
+        } else if other.upper.is_null() {
+            self.upper.clone()
+        } else {
+            min(&self.upper, &other.upper)?
+        };
+        Ok(if !lower.is_null() && !upper.is_null() && lower > upper {
+            // This None value signals an empty interval.
+            None
+        } else {
+            Some(Interval { lower, upper })
+        })
+    }
+
+    // Compute the negation of the interval.
+    #[allow(dead_code)]
+    pub(crate) fn arithmetic_negate(&self) -> Result<Interval> {
+        Ok(Interval {
+            lower: self.upper.arithmetic_negate()?,
+            upper: self.lower.arithmetic_negate()?,
+        })
+    }
+
+    /// Add the given interval (`other`) to this interval. Say we have
+    /// intervals [a1, b1] and [a2, b2], then their sum is [a1 + a2, b1 + b2].
+    /// Note that this represents all possible values the sum can take if
+    /// one can choose single values arbitrarily from each of the operands.
+    pub fn add<T: Borrow<Interval>>(&self, other: T) -> Result<Interval> {
+        let rhs = other.borrow();
+        let lower = if self.lower.is_null() || rhs.lower.is_null() {
+            ScalarValue::try_from(self.lower.get_datatype())
+        } else {
+            self.lower.add(&rhs.lower)
+        }?;
+        let upper = if self.upper.is_null() || rhs.upper.is_null() {
+            ScalarValue::try_from(self.upper.get_datatype())
+        } else {
+            self.upper.add(&rhs.upper)
+        }?;
+        Ok(Interval { lower, upper })
+    }
+
+    /// Subtract the given interval (`other`) from this interval. Say we have
+    /// intervals [a1, b1] and [a2, b2], then their sum is [a1 - b2, b1 - a2].
+    /// Note that this represents all possible values the difference can take
+    /// if one can choose single values arbitrarily from each of the operands.
+    pub fn sub<T: Borrow<Interval>>(&self, other: T) -> Result<Interval> {
+        let rhs = other.borrow();
+        let lower = if self.lower.is_null() || rhs.upper.is_null() {
+            ScalarValue::try_from(self.lower.get_datatype())
+        } else {
+            self.lower.sub(&rhs.upper)
+        }?;
+        let upper = if self.upper.is_null() || rhs.lower.is_null() {
+            ScalarValue::try_from(self.upper.get_datatype())
+        } else {
+            self.upper.sub(&rhs.lower)
+        }?;
+        Ok(Interval { lower, upper })
+    }
+}
+
+/// Indicates whether interval arithmetic is supported for the given operator.
+pub fn is_operator_supported(op: &Operator) -> bool {
+    matches!(
+        op,
+        &Operator::Plus
+            | &Operator::Minus
+            | &Operator::And
+            | &Operator::Gt

Review Comment:
   I think `LtEq` and `GtEq` should probably be covered (and would be naturally if we use the existing range analysis)



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ursabot commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ursabot (via GitHub)" <gi...@apache.org>.
ursabot commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1450748674

   Benchmark runs are scheduled for baseline = 03fbf9fecad00f8d6eb3e72e72ba16252b28b1d6 and contender = 3c1e4c0fd23012afa00dfd05a0571b61023b3d21. 3c1e4c0fd23012afa00dfd05a0571b61023b3d21 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/ddab7ca7b40a4ea28d10bdf4c4b4221b...6860bb40dadc4a6aa51a08c098b4dcd6/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on test-mac-arm] [test-mac-arm](https://conbench.ursa.dev/compare/runs/7a7fd81653a04f5a8c6b328022315aa1...f0aac3ff8f2348cfb979e0b27a57772e/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ursa-i9-9960x] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/45c2f3426a3b4a7e8ceb19ed88f6c207...47543a6a9ced41529445e0cf290033ab/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ursa-thinkcentre-m75q] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/94ce6c80d79b48f6b1bf11615ab5ee7b...d9b57a3ff53849058369f53b510cb346/)
   Buildkite builds:
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1441505615

   > Runtime Filters or Dynamic Filters(constructed dynamically during plan time), we also call this Join pushdown
   
   Does it delete the unrelated rows from the buffer (`RecordBatch` and Hash table) permanently or just filter the related rows for hash and match? Our approach makes the buffer constantly smaller.
   


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "mingmwang (via GitHub)" <gi...@apache.org>.
mingmwang commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1119865212


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   Good to know.  In the past, there was some discussion to enhance Ballista to support both BATCH/STREAMING execution models:
   
   https://docs.google.com/document/d/1OdAe078axk4qO0ozUxNqBMD4wKoBhzh9keMuLp_jerE/edit#
   http://www.vldb.org/pvldb/vol11/p746-yin.pdf
   I haven't working on Ballista since last year and there is no progress in this area.
   
   And in the latest Flink release, they had implement similar features(Bubble execution model, hybird shuffle etc).
   https://cwiki.apache.org/confluence/display/FLINK/FLIP-235%3A+Hybrid+Shuffle+Mode
   
   I think generally we can follow Flink's approach to make both DataFusion and Ballista support BATCH/STREAMING execution models.  In the high level, we can have different models(BATCH vs STREAMING), and user can specify the execution model. In the physical planing phase, we have `BatchPlanner` and `StreamingPlanner`, they can share some common rules, and batch and streaming planners can have their own rules. 
   In the `ExecutionPlan` trait, we can have another trait to indicate some operators are `Source` operators, the source operators can be `BOUNDED` or `UNBOUNDED`.    `BOUNDED` or `UNBOUNDED` should be a property available to Source operators only. 
   
   https://cwiki.apache.org/confluence/display/FLINK/FLIP-134%3A+Batch+execution+for+the+DataStream+API
   
   



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] metesynnada commented on pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#issuecomment-1445023954

   I am working on changes and I will update here soon.


-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120765946


##########
datafusion/physical-expr/src/utils.rs:
##########
@@ -235,6 +234,82 @@ pub fn ordering_satisfy_concrete<F: FnOnce() -> EquivalenceProperties>(
     }
 }
 
+#[derive(Clone, Debug)]
+pub struct ExprTreeNode<T> {

Review Comment:
   This comment still seems to be unaddressed. Perhaps we can handle it in a follow on PR? This one is already pretty large and been outstanding for a while



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] ozankabak commented on a diff in pull request #5322: Support for Sliding Windows Joins with Symmetric Hash Join (SHJ)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on code in PR #5322:
URL: https://github.com/apache/arrow-datafusion/pull/5322#discussion_r1120399841


##########
datafusion/core/src/physical_optimizer/pipeline_fixer.rs:
##########
@@ -182,13 +289,46 @@ fn apply_subrules_and_check_finiteness_requirements(
     physical_optimizer_subrules: &Vec<Box<PipelineFixerSubrule>>,
 ) -> Result<Option<PipelineStatePropagator>> {
     for sub_rule in physical_optimizer_subrules {
-        if let Some(value) = sub_rule(&input).transpose()? {
+        if let Some(value) = sub_rule(input.clone()).transpose()? {
             input = value;
         }
     }
     check_finiteness_requirements(input)
 }

Review Comment:
   We have made significant progress in foundational streaming execution support on the Datafusion side over the last few months. For example, you can already mark sources as infinite/unbounded (e.g. we have tests with FIFO files). Every `ExecutionPlan` then propagates this information upwards, so we make adjustments to the plan in an intelligent way depending on whether we are dealing with a stream.
   
   We will write a blog post about this soon, I will share with you when it is done so you can get up to speed on what we've done already. I think it would be also good to have a meeting with everyone interested in this to discuss future plans and designs.



-- 
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: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org