You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/03/09 14:54:31 UTC

[GitHub] [spark] somani opened a new pull request #35789: [WIP] Initial commit

somani opened a new pull request #35789:
URL: https://github.com/apache/spark/pull/35789


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r831290286



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,

Review comment:
       In [predicates.scala](https://github.com/apache/spark/blob/acb50d95a4952dea1cbbc27d4ddcc0b3432a13cf/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L128)




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyum commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824385275



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("bloom_filter_agg"),
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("might_contain"),
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  test("Test bloom_filter_agg and might_contain") {
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        BloomFilterAggregate.MAX_NUM_BITS)) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+                  |SELECT might_contain(cast(a as binary), cast(5 as long))
+                  |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "The Bloom filter binary input to might_contain should be either a constant value or " +
+        "a scalar subquery expression"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+                  |SELECT might_contain((select cast(a as binary)), cast(5 as long))
+                  |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "The Bloom filter binary input to might_contain should be either a constant value or " +
+        "a scalar subquery expression"))
+  }
+
+  test("Test that might_contain can take a constant value input") {
+    checkAnswer(spark.sql(
+      """SELECT might_contain(
+        |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267',
+        |cast(201 as long))""".stripMargin),
+      Row(false))
+  }
+
+  test("Test that bloom_filter_agg produces a NULL with empty input") {
+    checkAnswer(spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1)"""),
+      Row(null))
+  }
+
+  test("Test NULL inputs for might_contain") {
+    checkAnswer(spark.sql(
+      s"""
+         |SELECT might_contain(null, null) both_null,
+         |       might_contain(null, 1L) null_bf,
+         |       might_contain((SELECT bloom_filter_agg(cast(id as long)) from range(1, 10000)),
+         |            null) null_value
+         """.stripMargin),
+      Row(null, null, null))
+  }
+
+  test("Test that a query with bloom_filter_agg has partial aggregates") {
+    spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1000000)""")
+      .queryExecution.executedPlan.collect({case agg: BaseAggregateExec => agg}).size == 2

Review comment:
       Missing assert?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod edited a comment on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1064777368


   > Just not sure if this is in this PR's scope: the filter could be pushed down to table scan node[1],
   > thus filescan is also faster.
   
   The injected filter predicate is just a `BloomFilterMightContain` expression, which is nothing special from other expressions w.r.t. pushdown-ability. Filter pushdown is achieved by existing rules. Thus, if other filter expressions can be pushed down to a data source scan, it should also work for BloomFilterMightContain.
   
   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827288987



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max

Review comment:
       can we re-write this to :
   ```suggestion
       filterApplicationSide.collect({ 
         case leaf: LeafNode && leaf.stats.sizeInBytes != defaultSizeInBytes  => leaf.stats.sizeInBytes
       }).max
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r831292276



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,77 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =
+    buildConf("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a semi join in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_FILTER_NUMBER_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtimeFilter.number.threshold")
+      .doc("The total number of injected runtime filters (non-DPP) for a single " +
+        "query. This is to prevent driver OOMs with too many Bloom filters.")
+      .version("3.3.0")
+      .intConf
+      .checkValue(threshold => threshold >= 0, "The threshold should be >= 0")
+      .createWithDefault(10)
+
+  val RUNTIME_BLOOM_FILTER_ENABLED =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a bloom filter in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_BLOOM_FILTER_CREATION_SIDE_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.creationSideThreshold")
+      .doc("Size threshold of the bloom filter creation side plan. Estimated size needs to be " +
+        "under this value to try to inject bloom filter.")
+      .version("3.3.0")
+      .bytesConf(ByteUnit.BYTE)
+      .createWithDefaultString("10MB")
+
+  val RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizethreshold")
+      .doc("Byte size threshold of the Bloom filter application side plan's aggregated scan " +
+        "size. Aggregated scan byte size of the Bloom filter application side needs to be over " +
+        "this value to inject a bloom filter.")
+      .version("3.3.0")
+      .bytesConf(ByteUnit.BYTE)
+      .createWithDefaultString("10GB")
+
+  val RUNTIME_BLOOM_FILTER_EXPECTED_NUM_ITEMS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.expectedNumItems")
+      .doc("The default number of expected items for the runtime bloomfilter")
+      .version("3.3.0")
+      .longConf
+      .createWithDefault(1000000L)
+
+  val RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.maxNumItems")
+      .doc("The max allowed number of expected items for the runtime bloom filter")
+      .version("3.3.0")
+      .longConf
+      .createWithDefault(4000000L)
+
+
+  val RUNTIME_BLOOM_FILTER_NUM_BITS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.numBits")

Review comment:
       Yes this will be used if num items is not provided




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830874904



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {

Review comment:
       ```suggestion
     def hasBloomFilter(
         left: LogicalPlan,
         right: LogicalPlan,
         leftKey: Expression,
         rightKey: Expression): Boolean = {
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829247097



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }

Review comment:
       [question] consider a scenario where both the conf's BF as well as Semi join is enabled 
   
   1. when for the left child we called the injectFilter we tried BF and BF couldn't be applied where as Semi join could be but wasn't tried hence we marked no changes in left where as left was capable enough to have this Semi join reduction. 
   2.  consider the scenario where left we were not able to inject BF / SemiJoin so we turned to right child and tried apply BF which again couldn't be applied where SemiJoin was somehow, we could have missed applying semiJoin here as well.
   
   As per my understanding we will be missing case 1, 2 here. 
   Am I missing something here ? 




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829258472



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }

Review comment:
       [question] what would be the behaviour when both the conf's are enabled I believe we could structure this to something like : 
   ```suggestion
       val isRuntimeFilterPresent = false;
       if (conf.runtimeFilterBloomFilterEnabled) {
         isRuntimeFilterPresent = isRuntimeFilterPresent || hasBloomFilter(left, right, leftKey, rightKey)
       } 
       if (conf.runtimeFilterSemiJoinReductionEnabled) {
         isRuntimeFilterPresent = isRuntimeFilterPresent || hasInSubquery(left, right, leftKey, rightKey)
       }
       isRuntimeFilterPresent
   ```
   
   WDYT ? 




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] c21 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829414195



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       hmm @somani is it failing as part of maven build? Or some other unit test? We should exclude `BloomFilter.java` from check of backward compatibility, right? cc @cloud-fan.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1075827459


   thanks, merging to master/3.3!


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824850225



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  /**
+   * Returns whether an expression is likely to be selective
+   */
+  private def isLikelySelective(e: Expression): Boolean = e match {
+    case Not(expr) => isLikelySelective(expr)
+    case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+    case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+    case _: StringRegexExpression => true
+    case _: BinaryComparison => true
+    case _: In | _: InSet => true
+    case _: StringPredicate => true
+    case _: MultiLikeBase => true
+    case _ => false
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }

Review comment:
       Thanks! Deduplicated now.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] weixiuli commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
weixiuli commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824635657



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  /**
+   * Returns whether an expression is likely to be selective
+   */
+  private def isLikelySelective(e: Expression): Boolean = e match {
+    case Not(expr) => isLikelySelective(expr)
+    case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+    case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+    case _: StringRegexExpression => true
+    case _: BinaryComparison => true
+    case _: In | _: InSet => true
+    case _: StringPredicate => true
+    case _: MultiLikeBase => true
+    case _ => false
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+              filterCounter = filterCounter + 1
+            } else if (canFilterRight(joinType) && filteringHasBenefit(right, left, r, hint)) {
+              newRight = injectFilter(r, newRight, l, left)
+              filterCounter = filterCounter + 1
+            }
+          }
+        })
+        Join(newLeft, newRight, joinType, join.condition, hint)
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+    case s: Subquery if s.correlated => plan
+    case _ if !conf.runtimeFilterSemiJoinReductionEnabled &&
+      !conf.runtimeFilterBloomFilterEnabled => plan

Review comment:
       I'm interested in your PR, but why is this `&&`?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827199219



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       yea, this is not a public API and we don't need to worry about backward compatibility.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830875994



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)

Review comment:
       is it better to have a whitelist to define simple expressions?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830567530



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            val oldLeft = newLeft
+            val oldRight = newRight
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+            }
+            // Did we actually inject on the left? If not, try on the right
+            if (newLeft.fastEquals(oldLeft) && canFilterRight(joinType) &&
+              filteringHasBenefit(right, left, r, hint)) {
+              newRight = injectFilter(r, newRight, l, left)
+            }
+            if (!newLeft.fastEquals(oldLeft) || !newRight.fastEquals(oldRight)) {
+              filterCounter = filterCounter + 1
+            }
+          }
+        })
+        Join(newLeft, newRight, joinType, join.condition, hint)

Review comment:
       [minor] since we are just changing join's children should we do 
   ```scala
   join.withNewChildren(Seq(newLeft, newRight)) 
   ```
   as if there are some tags added to the node, we want it to be copied over in the new Node WDYT ? 




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830940054



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("bloom_filter_agg"),

Review comment:
       shall we call `dropFunction` in `afterAll` to unregister the functions at the end of this test suite?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828294228



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty

Review comment:
       Done!




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830861421



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,

Review comment:
       We can add a `def this` with `numBitsExpression: Expression` parameter, cast it to int literal and get the int value.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #35789:
URL: https://github.com/apache/spark/pull/35789


   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828279487



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,

Review comment:
       We use it in tests as SQL function (BloomFilterAggregateQuerySuite)

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private lazy val bloomFilter = {

Review comment:
       The bloom filter is calculated as a scalar subquery, so it is similar to broadcasting 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyuan commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
zhouyuan commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1064715855


   @sigmod did some quick tests on Q1 and it's working. :+1:  
   Just not sure if this is in this PR's scope: the filter could be pushed down to table scan node[1], thus filescan is also faster. 
   ![image](https://user-images.githubusercontent.com/43917/157791466-8442209a-8e1d-4276-bee6-2b81770a44cb.png)
   
   [1] [dynamic filter in presto](https://trino.io/docs/current/admin/dynamic-filtering.html)
   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyum commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824384048



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))

Review comment:
       Could we use [column statistics](https://github.com/apache/spark/blob/ddc61e62b9af5deff1b93e22f466f2a13f281155/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala#L95) if it is exist?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1064777368


   > Just not sure if this is in this PR's scope: the filter could be pushed down to table scan node[1],
   > thus filescan is also faster.
   
   The injected filter predicate is just a `BloomFilterMightContain` expression, which is nothing special from other expressions in terms pushdown-ability. Filter pushdown is achieved by existing rules. Thus, if other filter expressions can be pushed down to a data source scan, it should also work for BloomFilterMightContain.
   
   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] weixiuli commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
weixiuli commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r825561093



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)

Review comment:
       The require may be unnecessary, because they have been checked  before that in the tryInjectRuntimeFilter.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827276887



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {

Review comment:
       I'm not sure if this check helps. People can do a projection before join, to make the join keys simply attributes. However, when we push down the runtime filter and de-alias the keys, the final filter condition could get super expensive.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828292486



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))

Review comment:
       IIRC, as we discussed previously offline, `rowCount` is more conservative and safer? 
   Feel free to tune the sizing heuristics after this PR, if you saw benefits in your environments.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] jackylee-ch commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
jackylee-ch commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827559770



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {

Review comment:
       nit: the definition of typecheck is not necessary, we can return directly above. Futher more, `bloomFilterExpression ` check can also moved to upper place.
   ```
   case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
              (BinaryType, LongType) =>
       bloomFilterExpression match ...
   case _ =>
       TypeCheckResult.TypeCheckFailure(...)
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult

Review comment:
       nit: same as above.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827871685



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true

Review comment:
       Created https://issues.apache.org/jira/browse/SPARK-38565

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)

Review comment:
       Good catch. Fixed!




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828299017



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
##########
@@ -43,6 +43,8 @@ class SparkOptimizer(
     Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
     Batch("PartitionPruning", Once,
       PartitionPruning) :+
+    Batch("InjectRuntimeFilter", FixedPoint(1),

Review comment:
       I saw some idempotency check fail when I tested. I can try to do this in a follow up




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] weixiuli commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
weixiuli commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824635657



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  /**
+   * Returns whether an expression is likely to be selective
+   */
+  private def isLikelySelective(e: Expression): Boolean = e match {
+    case Not(expr) => isLikelySelective(expr)
+    case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+    case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+    case _: StringRegexExpression => true
+    case _: BinaryComparison => true
+    case _: In | _: InSet => true
+    case _: StringPredicate => true
+    case _: MultiLikeBase => true
+    case _ => false
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+              filterCounter = filterCounter + 1
+            } else if (canFilterRight(joinType) && filteringHasBenefit(right, left, r, hint)) {
+              newRight = injectFilter(r, newRight, l, left)
+              filterCounter = filterCounter + 1
+            }
+          }
+        })
+        Join(newLeft, newRight, joinType, join.condition, hint)
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+    case s: Subquery if s.correlated => plan
+    case _ if !conf.runtimeFilterSemiJoinReductionEnabled &&
+      !conf.runtimeFilterBloomFilterEnabled => plan

Review comment:
       ~~I'm interested in your PR, but why is this `&&`?~~




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] weixiuli commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
weixiuli commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r825561359



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    if (!estimatedNumItemsExpression.foldable) {
+      TypeCheckFailure("The estimated number of items provided must be a constant literal")
+    } else if (estimatedNumItems <= 0L) {
+      TypeCheckFailure("The estimated number of items must be a positive value " +
+        s" (current value = $estimatedNumItems)")
+    } else if (!numBitsExpression.foldable) {
+      TypeCheckFailure("The number of bits provided must be a constant literal")
+    } else if (numBits <= 0L) {
+      TypeCheckFailure("The number of bits must be a positive value " +
+        s" (current value = $numBits)")
+    } else {
+      require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+      require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+      TypeCheckSuccess
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_NUM_BITS)
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(newChild: Expression,
+      newEstimatedNumItemsExpression: Expression, newNumBitsExpression: Expression)
+  : BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serde.serialize(obj)
+  }
+
+  override def deserialize(bytes: Array[Byte]): BloomFilter = {
+    BloomFilterAggregate.serde.deserialize(bytes)
+  }
+}
+
+object BloomFilterAggregate {
+
+  val DEFAULT_EXPECTED_NUM_ITEMS: Long = 1000000L // Default 1M distinct items
+
+  val MAX_ALLOWED_NUM_ITEMS: Long = 4000000L // At most 4M distinct items
+
+  val DEFAULT_NUM_BITS: Long = 8388608 // Default 1MB
+
+  val MAX_NUM_BITS: Long = 67108864 // At most 8MB
+
+  /**
+   * Serializer/Deserializer for class [[BloomFilter]]
+   *
+   * This class is thread safe.
+   */
+  class BloomFilterSerDe {
+
+    final def serialize(obj: BloomFilter): Array[Byte] = {
+      val size = obj.bitSize()/8

Review comment:
       nit: a space




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] c21 commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1067524995


   cc @sunchao and @viirya FYI. You guys might be interested in reviewing 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] ulysses-you commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063664600


   > I have a question: why do we need Semi-Join if we have Bloom Filter?
   
   I guess it is a trade-off between benifits and costs. BloomFilter has false positives issue and it get worse if the data set is large. So if the creation side (from the design docs) is small enough which can be broadcast, we can use semi-join to get more benifits with less cost since it is accuracy. And It is something like dpp did.
   


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod edited a comment on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063664882


   > I have a question: why do we need Semi-Join if we have Bloom Filter?
   
   When Bloom filter is enabled and kicks in, Semi-Join is not needed. 
   It's under a config that is disabled by default, for tunings in the future, to on par. to the InSet filter in your original 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1064361243


   cc @cloud-fan


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyum commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r826486812



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {

Review comment:
       Could we format the code?
   ```diff
   --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
   +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
   @@ -70,8 +70,7 @@ object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with J
          filterApplicationSideExp: Expression,
          filterApplicationSidePlan: LogicalPlan,
          filterCreationSideExp: Expression,
   -      filterCreationSidePlan: LogicalPlan
   -  ): LogicalPlan = {
   +      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
        // Skip if the filter creation side is too big
        if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
          return filterApplicationSidePlan
   
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions

Review comment:
       Remove this line?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827270544



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private var bloomFilter: BloomFilter = _
+
+  override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {

Review comment:
       Codegen can also be supported easily
   ```
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     if (bloomFilter == null) {
       ev.copy(isNull = TrueLiteral, value = JavaCode.defaultLiteral(dataType))
     } else {
       val bf = ctx.addReferenceObj("bloomFilter", bloomFilter)
       val valueEval = valueExpression.genCode(ctx)
       val code = s"""
         boolean ${ev.isNull} = ${valueEval.isNull};
         ${ctx.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
         if (!${ev.isNull}) {
           ${ev.value} = $bf.mightContainLong(${valueEval.value});
         }
       """
     }
   }
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827261922



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private var bloomFilter: BloomFilter = _
+
+  override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {

Review comment:
       I think a better and simpler implementation is to take more control of the `eval` method and build the bloom filter ahead.
   ```
   private lazy val bloomFilter = {
     val bytes = bloomFilterExpression.eval().asInstanceOf[Array[Byte]]
     if (bytes == null) null else deserialize(bytes)
   }
   
   override def eval(input: InternalRow): Any = {
     if (bloomFilter == null) {
       null
     } else {
       val value = valueExpression.eval(input)
       if (value == null) null else bloomFilter.mightContainLong(value.asInstanceOf[Long])
     }
   }
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827247756



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty

Review comment:
       ```suggestion
       plan.collectFirst {
         case j @ Join(left, right, _, _, hint)
             if isProbablyShuffleJoin(left, right, hint) => j
         case a: Aggregate => a
       }.nonEmpty
   ```

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the

Review comment:
       [Nit]
   ```suggestion
       // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830880564



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)

Review comment:
       This is not a new item but part of the item above.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830937789



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,77 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =
+    buildConf("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a semi join in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_FILTER_NUMBER_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtimeFilter.number.threshold")
+      .doc("The total number of injected runtime filters (non-DPP) for a single " +
+        "query. This is to prevent driver OOMs with too many Bloom filters.")
+      .version("3.3.0")
+      .intConf
+      .checkValue(threshold => threshold >= 0, "The threshold should be >= 0")
+      .createWithDefault(10)
+
+  val RUNTIME_BLOOM_FILTER_ENABLED =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a bloom filter in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_BLOOM_FILTER_CREATION_SIDE_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.creationSideThreshold")
+      .doc("Size threshold of the bloom filter creation side plan. Estimated size needs to be " +
+        "under this value to try to inject bloom filter.")
+      .version("3.3.0")
+      .bytesConf(ByteUnit.BYTE)
+      .createWithDefaultString("10MB")
+
+  val RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizethreshold")
+      .doc("Byte size threshold of the Bloom filter application side plan's aggregated scan " +
+        "size. Aggregated scan byte size of the Bloom filter application side needs to be over " +
+        "this value to inject a bloom filter.")
+      .version("3.3.0")
+      .bytesConf(ByteUnit.BYTE)
+      .createWithDefaultString("10GB")
+
+  val RUNTIME_BLOOM_FILTER_EXPECTED_NUM_ITEMS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.expectedNumItems")
+      .doc("The default number of expected items for the runtime bloomfilter")
+      .version("3.3.0")
+      .longConf
+      .createWithDefault(1000000L)
+
+  val RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.maxNumItems")
+      .doc("The max allowed number of expected items for the runtime bloom filter")
+      .version("3.3.0")
+      .longConf
+      .createWithDefault(4000000L)
+
+
+  val RUNTIME_BLOOM_FILTER_NUM_BITS =
+    buildConf("spark.sql.optimizer.runtime.bloomFilter.numBits")

Review comment:
       hmm, doesn't the num bits depend on the num items?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829350854



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }

Review comment:
       You're right, but the idea was to not enable both at the same time, but provide both options for people to be able to  use one or the other based on their usecases. We plan to enable bloom filter based rewrite in a follow up PR, but keep semi join based rewrite disabled.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063624278


   cc @wangyum @c21


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyum commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
wangyum commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063648526


   I have a question: why do we need Semi-Join if we have Bloom Filter?


-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829247097



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }

Review comment:
       [question] consider a scenario where both the conf's BF as well as Semi join is enabled 
   
   1. when for the left child we called the injectFilter we tried BF and BF couldn't be applied where as Semi join could be but wasn't tried hence we marked no changes in left where as left was capable enough to have this Semi join reduction. 
   2.  consider the scenario where left we were not able to inject BF / SemiJoin so we turned to right child and tried apply BF which again couldn't be applied where SemiJoin was somehow, we could have missed applying semiJoin here as well.
   
   As per my understanding we will be missing case 1, 2 here. 
   Am I missing something here ? 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }

Review comment:
       [question] what would be the behaviour when both the conf's are enabled I believe we could structure this to something like : 
   ```suggestion
       val isRuntimeFilterPresent = false;
       if (conf.runtimeFilterBloomFilterEnabled) {
         isRuntimeFilterPresent = isRuntimeFilterPresent || hasBloomFilter(left, right, leftKey, rightKey)
       } 
       if (conf.runtimeFilterSemiJoinReductionEnabled) {
         isRuntimeFilterPresent = isRuntimeFilterPresent || hasInSubquery(left, right, leftKey, rightKey)
       }
       isRuntimeFilterPresent
   ```
   
   WDYT ? 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }

Review comment:
       [question] what would be the behaviour when both the conf's are enabled I believe we should check both the runtime filters presence then WDYT ? 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max

Review comment:
       can we re-write this to :
   ```suggestion
       filterApplicationSide.collect({ 
         case leaf: LeafNode && leaf.stats.sizeInBytes != defaultSizeInBytes  => leaf.stats.sizeInBytes
       }).max
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830000481



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```[error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")```
   
   https://github.com/somani/spark/runs/5569255844

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```[error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")
   ```
   
   
   
   https://github.com/somani/spark/runs/5569255844




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r831411116



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_EXPECTED_NUM_ITEMS)),
+      Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_NUM_BITS)))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) =>
+        if (!estimatedNumItemsExpression.foldable) {
+          TypeCheckFailure("The estimated number of items provided must be a constant literal")
+        } else if (estimatedNumItems <= 0L) {
+          TypeCheckFailure("The estimated number of items must be a positive value " +
+            s" (current value = $estimatedNumItems)")
+        } else if (!numBitsExpression.foldable) {
+          TypeCheckFailure("The number of bits provided must be a constant literal")
+        } else if (numBits <= 0L) {
+          TypeCheckFailure("The number of bits must be a positive value " +
+            s" (current value = $numBits)")
+        } else {
+          require(estimatedNumItems <=
+            SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))
+          require(numBits <= SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))
+          TypeCheckSuccess
+        }
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serialize(obj)
+  }
+
+  override def deserialize(bytes: Array[Byte]): BloomFilter = {
+    BloomFilterAggregate.deserialize(bytes)
+  }
+}
+
+object BloomFilterAggregate {
+  final def serialize(obj: BloomFilter): Array[Byte] = {
+    val size = obj.bitSize() / 8

Review comment:
       Makes sense, added 8 bytes to the size

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            val oldLeft = newLeft
+            val oldRight = newRight
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+            }
+            // Did we actually inject on the left? If not, try on the right
+            if (newLeft.fastEquals(oldLeft) && canFilterRight(joinType) &&
+              filteringHasBenefit(right, left, r, hint)) {
+              newRight = injectFilter(r, newRight, l, left)
+            }
+            if (!newLeft.fastEquals(oldLeft) || !newRight.fastEquals(oldRight)) {
+              filterCounter = filterCounter + 1
+            }
+          }
+        })
+        Join(newLeft, newRight, joinType, join.condition, hint)

Review comment:
       Makes sense, done!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)

Review comment:
       Do you mean configurable or hardcoded?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)

Review comment:
       Oops I meant to remove the code comment below... removed it now.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("bloom_filter_agg"),

Review comment:
       Done!




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r833425598



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  val funcId_bloom_filter_agg = new FunctionIdentifier("bloom_filter_agg")
+  val funcId_might_contain = new FunctionIdentifier("might_contain")
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_bloom_filter_agg,
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_might_contain,
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  override def afterAll(): Unit = {
+    FunctionRegistry.builtin.dropFunction(funcId_bloom_filter_agg)
+    FunctionRegistry.builtin.dropFunction(funcId_might_contain)
+    super.afterAll()
+  }
+
+  test("Test bloom_filter_agg and might_contain") {
+    val conf = SQLConf.get
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {

Review comment:
       `Numeric <=> Binary` is [not allowed in ansi mode](https://github.com/apache/spark/pull/30260),  so the content of  `exception.getMessage` is different from that of non-anis mode.
   




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] LuciferYang commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r833436767



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  val funcId_bloom_filter_agg = new FunctionIdentifier("bloom_filter_agg")
+  val funcId_might_contain = new FunctionIdentifier("might_contain")
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_bloom_filter_agg,
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_might_contain,
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  override def afterAll(): Unit = {
+    FunctionRegistry.builtin.dropFunction(funcId_bloom_filter_agg)
+    FunctionRegistry.builtin.dropFunction(funcId_might_contain)
+    super.afterAll()
+  }
+
+  test("Test bloom_filter_agg and might_contain") {
+    val conf = SQLConf.get
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {

Review comment:
       @HyukjinKwon try to  fix this https://github.com/apache/spark/pull/35953/files




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r831644893



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)

Review comment:
       >> is it better to have a whitelist to define simple expressions?
   
   Yes. However, 
   - if we use concrete expression classes for patterns, the whitelist will probably be too large;
   - if we use abstract expression classes for patterns, it may be unfriendly to future expressions like the negation approach.
   




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827278756



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {

Review comment:
       I think a cleaner idea is to always insert the runtime filter, then run the predicate pushdown batch, and finally remove non-beneficial runtime filters in the batch `"Cleanup filters that cannot be pushed down"`.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828003039



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    if (!estimatedNumItemsExpression.foldable) {
+      TypeCheckFailure("The estimated number of items provided must be a constant literal")
+    } else if (estimatedNumItems <= 0L) {
+      TypeCheckFailure("The estimated number of items must be a positive value " +
+        s" (current value = $estimatedNumItems)")
+    } else if (!numBitsExpression.foldable) {
+      TypeCheckFailure("The number of bits provided must be a constant literal")
+    } else if (numBits <= 0L) {
+      TypeCheckFailure("The number of bits must be a positive value " +
+        s" (current value = $numBits)")
+    } else {
+      require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+      require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+      TypeCheckSuccess
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_NUM_BITS)
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serde.serialize(obj)
+  }
+
+  override def deserialize(bytes: Array[Byte]): BloomFilter = {
+    BloomFilterAggregate.serde.deserialize(bytes)
+  }
+}
+
+object BloomFilterAggregate {
+
+  val DEFAULT_EXPECTED_NUM_ITEMS: Long = 1000000L // Default 1M distinct items
+
+  val MAX_ALLOWED_NUM_ITEMS: Long = 4000000L // At most 4M distinct items
+
+  val DEFAULT_NUM_BITS: Long = 8388608 // Default 1MB
+
+  val MAX_NUM_BITS: Long = 67108864 // At most 8MB

Review comment:
       Should we make these four configurable?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828038869



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private lazy val bloomFilter = {

Review comment:
       the bloomFilter maybe 50M~100M on our production system, what about broadcasting 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828581122



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    if (!estimatedNumItemsExpression.foldable) {
+      TypeCheckFailure("The estimated number of items provided must be a constant literal")
+    } else if (estimatedNumItems <= 0L) {
+      TypeCheckFailure("The estimated number of items must be a positive value " +
+        s" (current value = $estimatedNumItems)")
+    } else if (!numBitsExpression.foldable) {
+      TypeCheckFailure("The number of bits provided must be a constant literal")
+    } else if (numBits <= 0L) {
+      TypeCheckFailure("The number of bits must be a positive value " +
+        s" (current value = $numBits)")
+    } else {
+      require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+      require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+      TypeCheckSuccess
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_NUM_BITS)
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serde.serialize(obj)

Review comment:
       Removed.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    if (!estimatedNumItemsExpression.foldable) {
+      TypeCheckFailure("The estimated number of items provided must be a constant literal")
+    } else if (estimatedNumItems <= 0L) {
+      TypeCheckFailure("The estimated number of items must be a positive value " +
+        s" (current value = $estimatedNumItems)")
+    } else if (!numBitsExpression.foldable) {
+      TypeCheckFailure("The number of bits provided must be a constant literal")
+    } else if (numBits <= 0L) {
+      TypeCheckFailure("The number of bits must be a positive value " +
+        s" (current value = $numBits)")
+    } else {
+      require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+      require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+      TypeCheckSuccess
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_NUM_BITS)
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serde.serialize(obj)
+  }
+
+  override def deserialize(bytes: Array[Byte]): BloomFilter = {
+    BloomFilterAggregate.serde.deserialize(bytes)
+  }
+}
+
+object BloomFilterAggregate {
+
+  val DEFAULT_EXPECTED_NUM_ITEMS: Long = 1000000L // Default 1M distinct items
+
+  val MAX_ALLOWED_NUM_ITEMS: Long = 4000000L // At most 4M distinct items
+
+  val DEFAULT_NUM_BITS: Long = 8388608 // Default 1MB
+
+  val MAX_NUM_BITS: Long = 67108864 // At most 8MB

Review comment:
       Done




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] c21 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r826511449



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true

Review comment:
       it should work for `LEFT SEMI` join, right?

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       nit: why we need to provide a default implementation here, other than defining this as abstract method like others? 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)

Review comment:
       `satisfyByteSizeRequirement` only checks size of `filter application side`. it seems not sync with the comment above: 
   
   ```
   4. The filterApplicationSide is larger than the filterCreationSide by a configurable threshold
   ```
   
   Shouldn't we check size of `filterCreationSide` to be smaller enough as well here?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate

Review comment:
       do we mean `or a broadcast join that has a shuffle join or aggregate ...` based on implementation of `probablyHasShuffle()`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)

Review comment:
       hmm it seems that inside `injectFilter()`, we have additional logic to check filter creation side size to decide whether to inject the filter. Wouldn't it be the case that we can miss the filter injection opportunity below on right side branch, when `injectFilter()` does nothing on left side? (`else if` on line 277 below)

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:

Review comment:
       nit: better to move this comment to be a javadoc top-level comment (`/* ... */`) before this method.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount

Review comment:
       The size of bloom filter depends on number of rows here. Should we guard the logic to not inject bloom filter if number of rows being too large?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+              filterCounter = filterCounter + 1

Review comment:
       It seems to me it feels like an anti-pattern to update some variable inside `transformUp`. But feel free to leave it as it is if there's no better option.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private var bloomFilter: BloomFilter = _
+
+  override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {
+    if (bloomFilter == null) {
+      bloomFilter = deserialize(bloomFilterBytes.asInstanceOf[Array[Byte]])
+    }
+    bloomFilter.mightContainLong(value.asInstanceOf[Long])
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val thisObj = ctx.addReferenceObj("thisObj", this)
+    nullSafeCodeGen(ctx, ev, (bloomFilterBytes, value) => {
+      s"\n${ev.value} = (Boolean) $thisObj.nullSafeEval($bloomFilterBytes, $value);\n"

Review comment:
       It looks like we are just calling non-code-gen code inside code-gen code path. Why we cannot use `CodegenFallback` to start with? Or just provide code-gen implementation here?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,48 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =
+    buildConf("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a semi join in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_FILTER_NUMBER_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtimeFilter.number.threshold")
+      .doc("The total number of injected runtime filters (non-DPP) for a single " +
+        "query. This is to prevent driver OOMs with too many Bloom filters.")
+      .version("3.3.0")
+      .intConf
+      .checkValue(threshold => threshold >= 0, "The threshold should be >= 0")
+      .createWithDefault(10)
+
+  lazy val RUNTIME_BLOOM_FILTER_ENABLED =

Review comment:
       why this config needs to be `lazy val`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
##########
@@ -43,6 +43,8 @@ class SparkOptimizer(
     Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
     Batch("PartitionPruning", Once,
       PartitionPruning) :+
+    Batch("InjectRuntimeFilter", FixedPoint(1),

Review comment:
       curious why it's `FixedPoint(1)`, not `Once`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {

Review comment:
       I think the config name is a bit misleading. Either the config name should be changed to `runtimeFilterCreationSizeThreshold`, or the logic here to check the actual bloom filter size, instead of filter creation side size.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {

Review comment:
       I think it's ok to start with this heuristics, but I think it can be fragile for some queries. Cases like joining two bucketed tables would be regressed as the query plan normally has join operator, but does not have shuffle. Also it does not play very well with on-going [Storage Partitioned Join](https://issues.apache.org/jira/browse/SPARK-37375) work, where shuffle can be removed when joining on subset of join keys. But I don't think we have a good way to detect if the query plan has shuffle or not in logical plan phase. So this can be something to think about in the future.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,48 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =

Review comment:
       nit: given the feature is experimental and disable by default now. It would be better to mark these configs to be `.internal()`.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828286164



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {

Review comment:
       Makes sense, will change!




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830000481



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```
   [error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")
   ```
   
   
   
   https://github.com/somani/spark/runs/5569255844




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] c21 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829414195



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       hmm @somani is it failing as part of maven build? Or some other unit test? We should exclude `BloomFilter.java` from check of backward compatibility, right? cc @cloud-fan.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828021703



##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Turns out BloomFilter is public, and removing this caused backward compatibility tests to fail.
   
   ```
   [error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")
   ```
   
   So added this back 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830858774



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, JavaCode, TrueLiteral}
+import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) =>
+        bloomFilterExpression match {
+          case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+          case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+            TypeCheckResult.TypeCheckSuccess
+          case _ =>
+            TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+              "should be either a constant value or a scalar subquery expression")
+        }
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private lazy val bloomFilter = {
+    val bytes = bloomFilterExpression.eval().asInstanceOf[Array[Byte]]
+    if (bytes == null) null else deserialize(bytes)
+  }
+
+  override def eval(input: InternalRow): Any = {
+    if (bloomFilter == null) {
+      null

Review comment:
       if bloom filter is null, we should return true?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830430230



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_EXPECTED_NUM_ITEMS)),
+      Literal(SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_NUM_BITS)))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) =>
+        if (!estimatedNumItemsExpression.foldable) {
+          TypeCheckFailure("The estimated number of items provided must be a constant literal")
+        } else if (estimatedNumItems <= 0L) {
+          TypeCheckFailure("The estimated number of items must be a positive value " +
+            s" (current value = $estimatedNumItems)")
+        } else if (!numBitsExpression.foldable) {
+          TypeCheckFailure("The number of bits provided must be a constant literal")
+        } else if (numBits <= 0L) {
+          TypeCheckFailure("The number of bits must be a positive value " +
+            s" (current value = $numBits)")
+        } else {
+          require(estimatedNumItems <=
+            SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))
+          require(numBits <= SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))
+          TypeCheckSuccess
+        }
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serialize(obj)
+  }
+
+  override def deserialize(bytes: Array[Byte]): BloomFilter = {
+    BloomFilterAggregate.deserialize(bytes)
+  }
+}
+
+object BloomFilterAggregate {
+  final def serialize(obj: BloomFilter): Array[Byte] = {
+    val size = obj.bitSize() / 8

Review comment:
       `BloomFilterImpl.writeTo` need to write `VersionNumber` and `numHashFunctions` before `bits`, so this `size` is not enough as a initialSize hint. 




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830876869



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            val oldLeft = newLeft
+            val oldRight = newRight

Review comment:
       aren't these two just `left` and `right`?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r833176935



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  val funcId_bloom_filter_agg = new FunctionIdentifier("bloom_filter_agg")
+  val funcId_might_contain = new FunctionIdentifier("might_contain")
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_bloom_filter_agg,
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_might_contain,
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  override def afterAll(): Unit = {
+    FunctionRegistry.builtin.dropFunction(funcId_bloom_filter_agg)
+    FunctionRegistry.builtin.dropFunction(funcId_might_contain)
+    super.afterAll()
+  }
+
+  test("Test bloom_filter_agg and might_contain") {
+    val conf = SQLConf.get
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {

Review comment:
       This test seems failing with ANSI mode on:
   
   ```
   org.scalatest.exceptions.TestFailedException: "cannot resolve 'CAST(t.a AS BINARY)' due to data type mismatch: 
    cannot cast bigint to binary with ANSI mode on.
    If you have to cast bigint to binary, you can set spark.sql.ansi.enabled as false.
   ; line 2 pos 21;
   'Project [unresolvedalias('might_contain(cast(a#785367L as binary), cast(5 as bigint)), None)]
   +- SubqueryAlias t
      +- LocalRelation [a#785367L]
   " did not contain "The Bloom filter binary input to might_contain should be either a constant value or a scalar subquery expression"
   ```
   
   https://github.com/apache/spark/runs/5658074902




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830891604



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,

Review comment:
       where is this method defined?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhengruifeng commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
zhengruifeng commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828002421



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    if (!estimatedNumItemsExpression.foldable) {
+      TypeCheckFailure("The estimated number of items provided must be a constant literal")
+    } else if (estimatedNumItems <= 0L) {
+      TypeCheckFailure("The estimated number of items must be a positive value " +
+        s" (current value = $estimatedNumItems)")
+    } else if (!numBitsExpression.foldable) {
+      TypeCheckFailure("The number of bits provided must be a constant literal")
+    } else if (numBits <= 0L) {
+      TypeCheckFailure("The number of bits must be a positive value " +
+        s" (current value = $numBits)")
+    } else {
+      require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+      require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+      TypeCheckSuccess
+    }
+  }
+  override def nullable: Boolean = true
+
+  override def dataType: DataType = BinaryType
+
+  override def prettyName: String = "bloom_filter_agg"
+
+  // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+  private lazy val estimatedNumItems: Long =
+    Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+  // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+  private lazy val numBits: Long =
+    Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+      BloomFilterAggregate.MAX_NUM_BITS)
+
+  override def first: Expression = child
+
+  override def second: Expression = estimatedNumItemsExpression
+
+  override def third: Expression = numBitsExpression
+
+  override protected def withNewChildrenInternal(
+      newChild: Expression,
+      newEstimatedNumItemsExpression: Expression,
+      newNumBitsExpression: Expression): BloomFilterAggregate = {
+    copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+      numBitsExpression = newNumBitsExpression)
+  }
+
+  override def createAggregationBuffer(): BloomFilter = {
+    BloomFilter.create(estimatedNumItems, numBits)
+  }
+
+  override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+    val value = child.eval(inputRow)
+    // Ignore null values.
+    if (value == null) {
+      return buffer
+    }
+    buffer.putLong(value.asInstanceOf[Long])
+    buffer
+  }
+
+  override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+    buffer.mergeInPlace(other)
+  }
+
+  override def eval(buffer: BloomFilter): Any = {
+    if (buffer.cardinality() == 0) {
+      // There's no set bit in the Bloom filter and hence no not-null value is processed.
+      return null
+    }
+    serialize(buffer)
+  }
+
+  override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(mutableAggBufferOffset = newOffset)
+
+  override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+    copy(inputAggBufferOffset = newOffset)
+
+  override def serialize(obj: BloomFilter): Array[Byte] = {
+    BloomFilterAggregate.serde.serialize(obj)

Review comment:
       Do we need a new class `BloomFilterSerDe`?




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830880331



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)

Review comment:
       ```suggestion
      * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
      *      expression references originate from a single leaf node)
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830881324



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)

Review comment:
       Maybe we can remove the method doc as it just duplicates the code comment below




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] wangyum commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
wangyum commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824384997



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  /**
+   * Returns whether an expression is likely to be selective
+   */
+  private def isLikelySelective(e: Expression): Boolean = e match {
+    case Not(expr) => isLikelySelective(expr)
+    case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+    case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+    case _: StringRegexExpression => true
+    case _: BinaryComparison => true
+    case _: In | _: InSet => true
+    case _: StringPredicate => true
+    case _: MultiLikeBase => true
+    case _ => false
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }

Review comment:
       Duplicate code:
   https://github.com/apache/spark/blob/2da0d0722d5a0aa337d5744a51075449b224e421/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala#L197-L241




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824850347



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("bloom_filter_agg"),
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(new FunctionIdentifier("might_contain"),
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  test("Test bloom_filter_agg and might_contain") {
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        BloomFilterAggregate.MAX_NUM_BITS)) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+                  |SELECT might_contain(cast(a as binary), cast(5 as long))
+                  |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "The Bloom filter binary input to might_contain should be either a constant value or " +
+        "a scalar subquery expression"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+                  |SELECT might_contain((select cast(a as binary)), cast(5 as long))
+                  |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "The Bloom filter binary input to might_contain should be either a constant value or " +
+        "a scalar subquery expression"))
+  }
+
+  test("Test that might_contain can take a constant value input") {
+    checkAnswer(spark.sql(
+      """SELECT might_contain(
+        |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267',
+        |cast(201 as long))""".stripMargin),
+      Row(false))
+  }
+
+  test("Test that bloom_filter_agg produces a NULL with empty input") {
+    checkAnswer(spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1)"""),
+      Row(null))
+  }
+
+  test("Test NULL inputs for might_contain") {
+    checkAnswer(spark.sql(
+      s"""
+         |SELECT might_contain(null, null) both_null,
+         |       might_contain(null, 1L) null_bf,
+         |       might_contain((SELECT bloom_filter_agg(cast(id as long)) from range(1, 10000)),
+         |            null) null_value
+         """.stripMargin),
+      Row(null, null, null))
+  }
+
+  test("Test that a query with bloom_filter_agg has partial aggregates") {
+    spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1000000)""")
+      .queryExecution.executedPlan.collect({case agg: BaseAggregateExec => agg}).size == 2

Review comment:
       Thanks! Fixed.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] weixiuli commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
weixiuli commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r824635657



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  /**
+   * Returns whether an expression is likely to be selective
+   */
+  private def isLikelySelective(e: Expression): Boolean = e match {
+    case Not(expr) => isLikelySelective(expr)
+    case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+    case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+    case _: StringRegexExpression => true
+    case _: BinaryComparison => true
+    case _: In | _: InSet => true
+    case _: StringPredicate => true
+    case _: MultiLikeBase => true
+    case _ => false
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {
+            if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+              newLeft = injectFilter(l, newLeft, r, right)
+              filterCounter = filterCounter + 1
+            } else if (canFilterRight(joinType) && filteringHasBenefit(right, left, r, hint)) {
+              newRight = injectFilter(r, newRight, l, left)
+              filterCounter = filterCounter + 1
+            }
+          }
+        })
+        Join(newLeft, newRight, joinType, join.condition, hint)
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+    case s: Subquery if s.correlated => plan
+    case _ if !conf.runtimeFilterSemiJoinReductionEnabled &&
+      !conf.runtimeFilterBloomFilterEnabled => plan

Review comment:
       ~~I'm interested in your PR, but why is this `&&`?~~




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063664882


   > I have a question: why do we need Semi-Join if we have Bloom Filter?
   
   When Bloom filter is enabled, Semi-Join might not be needed. 
   It's under a config that is disabled by default, for tunings in the future, to on par. to the InSet filter in your original 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod edited a comment on pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod edited a comment on pull request #35789:
URL: https://github.com/apache/spark/pull/35789#issuecomment-1063664882


   > I have a question: why do we need Semi-Join if we have Bloom Filter?
   
   When Bloom filter is enabled and kicks in, Semi-Join is be needed. 
   It's under a config that is disabled by default, for tunings in the future, to on par. to the InSet filter in your original 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827174853



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {

Review comment:
       Done!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions

Review comment:
       Yes, thanks!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)

Review comment:
       Filter creation side has its own threshold. Maybe I should change the comment to
   > The max filterApplicationSide scan size is greater than a configurable threshold

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {

Review comment:
       Done!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,48 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =
+    buildConf("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled")
+      .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+        "to insert a semi join in the other side to reduce the amount of shuffle data.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  val RUNTIME_FILTER_NUMBER_THRESHOLD =
+    buildConf("spark.sql.optimizer.runtimeFilter.number.threshold")
+      .doc("The total number of injected runtime filters (non-DPP) for a single " +
+        "query. This is to prevent driver OOMs with too many Bloom filters.")
+      .version("3.3.0")
+      .intConf
+      .checkValue(threshold => threshold >= 0, "The threshold should be >= 0")
+      .createWithDefault(10)
+
+  lazy val RUNTIME_BLOOM_FILTER_ENABLED =

Review comment:
       It doesn't, changed. Thanks!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -341,6 +341,48 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =

Review comment:
       Umm, ill leave it for others to decide, but I think internal might be for internal configs that might just be used from within code, not for features that are experimental and are open for people to play with.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate

Review comment:
       Changed to: 
   `The current join is a shuffle join or a broadcast join that has a shuffle below it`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount

Review comment:
       We have a check against that in BloomFilterAggregate
   
   > `// Mark as lazy so that estimatedNumItems is not evaluated during tree transformation.
   >   private lazy val estimatedNumItems: Long =
   >     Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
   >       BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
   > 
   >   // Mark as lazy so that numBits is not evaluated during tree transformation.
   >   private lazy val numBits: Long =1
   >     Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
   >       BloomFilterAggregate.MAX_NUM_BITS) 

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private var bloomFilter: BloomFilter = _
+
+  override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {

Review comment:
       Done, thanks!

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+    bloomFilterExpression: Expression,
+    valueExpression: Expression) extends BinaryExpression {
+
+  override def nullable: Boolean = true
+  override def left: Expression = bloomFilterExpression
+  override def right: Expression = valueExpression
+  override def prettyName: String = "might_contain"
+  override def dataType: DataType = BooleanType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (left.dataType, right.dataType) match {
+      case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+           (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+        s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult
+    }
+    bloomFilterExpression match {
+      case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+      case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+        TypeCheckResult.TypeCheckSuccess
+      case _ =>
+        TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+          "should be either a constant value or a scalar subquery expression")
+    }
+  }
+
+  override protected def withNewChildrenInternal(
+      newBloomFilterExpression: Expression,
+      newValueExpression: Expression): BloomFilterMightContain =
+    copy(bloomFilterExpression = newBloomFilterExpression,
+      valueExpression = newValueExpression)
+
+  // The bloom filter created from `bloomFilterExpression`.
+  @transient private var bloomFilter: BloomFilter = _
+
+  override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {
+    if (bloomFilter == null) {
+      bloomFilter = deserialize(bloomFilterBytes.asInstanceOf[Array[Byte]])
+    }
+    bloomFilter.mightContainLong(value.asInstanceOf[Long])
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+    val thisObj = ctx.addReferenceObj("thisObj", this)
+    nullSafeCodeGen(ctx, ev, (bloomFilterBytes, value) => {
+      s"\n${ev.value} = (Boolean) $thisObj.nullSafeEval($bloomFilterBytes, $value);\n"

Review comment:
       Done!

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Makse sense, will change

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true

Review comment:
       Yes, let me raise a follow up for this.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+    //    threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+        pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+      case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+        pruningKey.fastEquals(rightKey) ||
+          hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+      case _ => false
+    }
+  }
+
+  def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+  }
+
+  private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+    plan.find {
+      case Filter(condition, _) =>
+        splitConjunctivePredicates(condition).exists {
+          case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+            if valueExpression.fastEquals(key) => true
+          case _ => false
+        }
+      case _ => false
+    }.isDefined
+  }
+
+  def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    (left, right) match {
+      case (Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+        key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+      case (_, Filter(InSubquery(Seq(key),
+      ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+        key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+      case _ => false
+    }
+  }
+
+  private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+    var filterCounter = 0
+    val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+    plan transformUp {
+      case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, left, right, hint) =>
+        var newLeft = left
+        var newRight = right
+        (leftKeys, rightKeys).zipped.foreach((l, r) => {
+          // Check if:
+          // 1. There is already a DPP filter on the key
+          // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+          // 3. The keys are simple cheap expressions
+          if (filterCounter < numFilterThreshold &&
+            !hasDynamicPruningSubquery(left, right, l, r) &&
+            !hasRuntimeFilter(newLeft, newRight, l, r) &&
+            isSimpleExpression(l) && isSimpleExpression(r)) {

Review comment:
       In isSelectiveFilterOverScan(), we also check that the filter creation side plan is a simple plan(project->filter->scan), and we check that all the filters themselves are made of simple expressions as well.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:

Review comment:
       Done




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827271638



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,

Review comment:
       This is not a real SQL function, I think we can use `Long` as the parameter type directly.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r828286474



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child                     Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression         The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+    child: Expression,
+    estimatedNumItemsExpression: Expression,
+    numBitsExpression: Expression,
+    override val mutableAggBufferOffset: Int,
+    override val inputAggBufferOffset: Int)
+  extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression,
+      numBitsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+  }
+
+  def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+    this(child, estimatedNumItemsExpression,
+      // 1 byte per item.
+      Multiply(estimatedNumItemsExpression, Literal(8L)))
+  }
+
+  def this(child: Expression) = {
+    this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+      Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+  }
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+      case (_, NullType, _) | (_, _, NullType) =>
+        TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+      case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+      case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+        s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+        s"arguments, but it's [${first.dataType.catalogString}, " +
+        s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+    }
+    if (typeCheckResult.isFailure) {
+      return typeCheckResult

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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r827288987



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collect {
+      case j@Join(left, right, _, _, hint)
+        if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+          !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max

Review comment:
       can we re-write this to :
   ```suggestion
       filterApplicationSide.collect({ 
         case leaf: LeafNode && leaf.stats.sizeInBytes != defaultSizeInBytes  => leaf.stats.sizeInBytes
       }).max
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
sigmod commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r825637184



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize >  IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)

Review comment:
       > because they have been checked before that in the tryInjectRuntimeFilter.
   
   Call sites of this function can evolve after this PR. `require` (like other `requires` in the codebase) is to prevent undesirable (potentially buggy) call site changes in the future.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r832365608



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {

Review comment:
       ```suggestion
         filterCreationSidePlan: LogicalPlan): LogicalPlan = {
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] singhpk234 commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829258472



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }

Review comment:
       [question] what would be the behaviour when both the conf's are enabled I believe we should check both the runtime filters presence then WDYT ? 




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r829350854



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }

Review comment:
       You're right, but the idea was to not enable both at the same time, but provide both options for people to be able to  use one or the other based on their usecases. We plan to enable bloom filter based rewrite in a follow up PR, but keep semi join based rewrite disabled.

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```[error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")```
   
   https://github.com/somani/spark/runs/5569255844

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```[error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")
   ```
   
   
   
   https://github.com/somani/spark/runs/5569255844

##########
File path: common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
##########
@@ -163,6 +163,13 @@ int getVersionNumber() {
    */
   public abstract void writeTo(OutputStream out) throws IOException;
 
+  /**
+   * @return the number of set bits in this {@link BloomFilter}.
+   */
+  public long cardinality() {
+    throw new UnsupportedOperationException("Not implemented");

Review comment:
       Yes regular sbt builds failed with
   
   ```
   [error] spark-sketch: Failed binary compatibility check against org.apache.spark:spark-sketch_2.12:3.2.0! Found 1 potential problems (filtered 1)
   [error]  * abstract method cardinality()Long in class org.apache.spark.util.sketch.BloomFilter is present only in current version
   [error]    filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.util.sketch.BloomFilter.cardinality")
   ```
   
   
   
   https://github.com/somani/spark/runs/5569255844




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r833258620



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  val funcId_bloom_filter_agg = new FunctionIdentifier("bloom_filter_agg")
+  val funcId_might_contain = new FunctionIdentifier("might_contain")
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_bloom_filter_agg,
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_might_contain,
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  override def afterAll(): Unit = {
+    FunctionRegistry.builtin.dropFunction(funcId_bloom_filter_agg)
+    FunctionRegistry.builtin.dropFunction(funcId_might_contain)
+    super.afterAll()
+  }
+
+  test("Test bloom_filter_agg and might_contain") {
+    val conf = SQLConf.get
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+                  |SELECT might_contain(cast(a as binary), cast(5 as long))

Review comment:
       to avoid ansi failure, we can do `cast(cast(a as string) as binary)`




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r833176935



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.FunctionIdentifier
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.aggregate.BaseAggregateExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+
+/**
+ * Query tests for the Bloom filter aggregate and filter function.
+ */
+class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession {
+  import testImplicits._
+
+  val funcId_bloom_filter_agg = new FunctionIdentifier("bloom_filter_agg")
+  val funcId_might_contain = new FunctionIdentifier("might_contain")
+
+  // Register 'bloom_filter_agg' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_bloom_filter_agg,
+    new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"),
+    (children: Seq[Expression]) => children.size match {
+      case 1 => new BloomFilterAggregate(children.head)
+      case 2 => new BloomFilterAggregate(children.head, children(1))
+      case 3 => new BloomFilterAggregate(children.head, children(1), children(2))
+    })
+
+  // Register 'might_contain' to builtin.
+  FunctionRegistry.builtin.registerFunction(funcId_might_contain,
+    new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"),
+    (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1)))
+
+  override def afterAll(): Unit = {
+    FunctionRegistry.builtin.dropFunction(funcId_bloom_filter_agg)
+    FunctionRegistry.builtin.dropFunction(funcId_might_contain)
+    super.afterAll()
+  }
+
+  test("Test bloom_filter_agg and might_contain") {
+    val conf = SQLConf.get
+    val table = "bloom_filter_test"
+    for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) {
+      for (numBits <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue,
+        conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS))) {
+        val sqlString = s"""
+                           |SELECT every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM $table),
+                           |            col)) positive_membership_test,
+                           |       every(might_contain(
+                           |            (SELECT bloom_filter_agg(col,
+                           |              cast($numEstimatedItems as long),
+                           |              cast($numBits as long))
+                           |             FROM values (-1L), (100001L), (20000L) as t(col)),
+                           |            col)) negative_membership_test
+                           |FROM $table
+           """.stripMargin
+        withTempView(table) {
+          (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L))
+            .toDF("col").createOrReplaceTempView(table)
+          // Validate error messages as well as answers when there's no error.
+          if (numEstimatedItems <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains(
+              "The estimated number of items must be a positive value"))
+          } else if (numBits <= 0) {
+            val exception = intercept[AnalysisException] {
+              spark.sql(sqlString)
+            }
+            assert(exception.getMessage.contains("The number of bits must be a positive value"))
+          } else {
+            checkAnswer(spark.sql(sqlString), Row(true, false))
+          }
+        }
+      }
+    }
+  }
+
+  test("Test that bloom_filter_agg errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a)
+        |FROM values (1.2), (2.5) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function bloom_filter_agg should have been a bigint value"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 2)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception3 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, cast(2 as long), 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception3.getMessage.contains(
+      "function bloom_filter_agg should have been a bigint value followed with two bigint"))
+
+    val exception4 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, null, 5)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception4.getMessage.contains("Null typed values cannot be used as size arguments"))
+
+    val exception5 = intercept[AnalysisException] {
+      spark.sql("""
+        |SELECT bloom_filter_agg(a, 5, null)
+        |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)"""
+        .stripMargin)
+    }
+    assert(exception5.getMessage.contains("Null typed values cannot be used as size arguments"))
+  }
+
+  test("Test that might_contain errors out disallowed input value types") {
+    val exception1 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(1.0, 1L)"""
+        .stripMargin)
+    }
+    assert(exception1.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+
+    val exception2 = intercept[AnalysisException] {
+      spark.sql("""|SELECT might_contain(NULL, 0.1)"""
+        .stripMargin)
+    }
+    assert(exception2.getMessage.contains(
+      "Input to function might_contain should have been binary followed by a value with bigint"))
+  }
+
+  test("Test that might_contain errors out non-constant Bloom filter") {

Review comment:
       This test seems flaky:
   
   ```
   org.scalatest.exceptions.TestFailedException: "cannot resolve 'CAST(t.a AS BINARY)' due to data type mismatch: 
    cannot cast bigint to binary with ANSI mode on.
    If you have to cast bigint to binary, you can set spark.sql.ansi.enabled as false.
   ; line 2 pos 21;
   'Project [unresolvedalias('might_contain(cast(a#785367L as binary), cast(5 as bigint)), None)]
   +- SubqueryAlias t
      +- LocalRelation [a#785367L]
   " did not contain "The Bloom filter binary input to might_contain should be either a constant value or a scalar subquery expression"
   ```
   
   https://github.com/apache/spark/runs/5658074902




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830879931



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the

Review comment:
       unfinished 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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830873689



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterCreationSideThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+    plan.collectFirst {
+      case j@Join(left, right, _, _, hint)
+        if isProbablyShuffleJoin(left, right, hint) => j
+      case a: Aggregate => a
+    }.nonEmpty
+  }
+
+  // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+  private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+    val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+    filterApplicationSide.collect({
+      case leaf: LeafNode => leaf
+    }).map(scan => {
+      // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+      // creating a Bloom filter when the filter application side is very small, so using 0
+      // as the byte size when the actual size is unknown can avoid regression by applying BF
+      // on a small table.
+      if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+    }).max
+  }
+
+  // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+  // Bloom filter; false otherwise.
+  private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+    // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+    // filter to each small task might be more costly than scanning them itself. Thus, we use max
+    // rather than sum here.
+    val maxScanSize = maxScanByteSize(filterApplicationSide)
+    maxScanSize >=
+      conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+  }
+
+  /**
+   * Check that:
+   * - The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+   * - expression references originate from a single leaf node)
+   * - The filter creation side has a selective predicate
+   * - The current join is a shuffle join or a broadcast join that has a shuffle below it
+   * - The max filterApplicationSide scan size is greater than a configurable threshold
+   */
+  private def filteringHasBenefit(
+      filterApplicationSide: LogicalPlan,
+      filterCreationSide: LogicalPlan,
+      filterApplicationSideExp: Expression,
+      hint: JoinHint): Boolean = {
+    // Check that:
+    // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (i.e the
+    //    expression references originate from a single leaf node)
+    // 2. The filter creation side has a selective predicate
+    // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+    //    in the filter application side
+    // 4. The max filterApplicationSide scan size is greater than a configurable threshold
+    findExpressionAndTrackLineageDown(filterApplicationSideExp,
+      filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+      (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+        probablyHasShuffle(filterApplicationSide)) &&
+      satisfyByteSizeRequirement(filterApplicationSide)
+  }
+
+  def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      hasBloomFilter(left, right, leftKey, rightKey)
+    } else {
+      hasInSubquery(left, right, leftKey, rightKey)
+    }
+  }
+
+  // This checks if there is already a DPP filter, as this rule is called just after DPP.
+  def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+      rightKey: Expression): Boolean = {

Review comment:
       ```suggestion
     def hasDynamicPruningSubquery(
         left: LogicalPlan,
         right: LogicalPlan,
         leftKey: Expression,
         rightKey: Expression): Boolean = {
   ```




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35789: [SPARK-32268][SQL] Row-level Runtime Filtering

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35789:
URL: https://github.com/apache/spark/pull/35789#discussion_r830886853



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+  // Wraps `expr` with a hash function if its byte size is larger than an integer.
+  private def mayWrapWithHash(expr: Expression): Expression = {
+    if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+      new Murmur3Hash(Seq(expr))
+    } else {
+      expr
+    }
+  }
+
+  private def injectFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+    if (conf.runtimeFilterBloomFilterEnabled) {
+      injectBloomFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    } else {
+      injectInSubqueryFilter(
+        filterApplicationSideExp,
+        filterApplicationSidePlan,
+        filterCreationSideExp,
+        filterCreationSidePlan
+      )
+    }
+  }
+
+  private def injectBloomFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan
+  ): LogicalPlan = {
+    // Skip if the filter creation side is too big
+    if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+      return filterApplicationSidePlan
+    }
+    val rowCount = filterCreationSidePlan.stats.rowCount
+    val bloomFilterAgg =
+      if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+          Literal(rowCount.get.longValue))
+      } else {
+        new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+      }
+    val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+    val alias = Alias(aggExp, "bloomFilter")()
+    val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+    val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+    val filter = BloomFilterMightContain(bloomFilterSubquery,
+      new XxHash64(Seq(filterApplicationSideExp)))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  private def injectInSubqueryFilter(
+      filterApplicationSideExp: Expression,
+      filterApplicationSidePlan: LogicalPlan,
+      filterCreationSideExp: Expression,
+      filterCreationSidePlan: LogicalPlan): LogicalPlan = {
+    require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+    val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+    val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+    val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+    if (!canBroadcastBySize(aggregate, conf)) {
+      // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+      // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+      return filterApplicationSidePlan
+    }
+    val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+      ListQuery(aggregate, childOutputs = aggregate.output))
+    Filter(filter, filterApplicationSidePlan)
+  }
+
+  /**
+   * Returns whether the plan is a simple filter over scan and the filter is likely selective
+   * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+   * do not add a subquery that might have an expensive computation
+   */
+  private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+    plan.expressions
+    val ret = plan match {
+      case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+        filters.forall(isSimpleExpression) &&
+          filters.exists(isLikelySelective)
+      case _ => false
+    }
+    !plan.isStreaming && ret
+  }
+
+  private def isSimpleExpression(e: Expression): Boolean = {
+    !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+      REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+  }
+
+  private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+    case Inner | RightOuter => true
+    case _ => false
+  }
+
+  private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+    case Inner | LeftOuter => true
+    case _ => false
+  }
+
+  private def isProbablyShuffleJoin(left: LogicalPlan,
+      right: LogicalPlan, hint: JoinHint): Boolean = {
+    !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+      !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+  }
+
+  private def probablyHasShuffle(plan: LogicalPlan): Boolean = {

Review comment:
       Yea, a possible idea is to always generate the bloom filter, and then remove non-beneficial ones at the physical phase.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org