You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ron8hu <gi...@git.apache.org> on 2017/03/24 20:54:11 UTC

[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

GitHub user ron8hu opened a pull request:

    https://github.com/apache/spark/pull/17415

    [SPARK-19408][SQL] filter estimation on two columns of same table

    ## What changes were proposed in this pull request?
    
    In SQL queries, we also see predicate expressions involving two columns such as "column-1 (op) column-2" where column-1 and column-2 belong to same table. Note that, if column-1 and column-2 belong to different tables, then it is a join operator's work, NOT a filter operator's work.  This PR estimates filter selectivity on two columns of same table.
    
    ## How was this patch tested?
    
    We added 6 new test cases to test various logical predicates involving two columns of same table.
    
    Please review http://spark.apache.org/contributing.html before opening a pull request.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ron8hu/spark filterTwoColumns

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/17415.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #17415
    
----
commit 893066905b690c78a127eae58b908dff1dabf7cf
Author: Ron Hu <ro...@huawei.com>
Date:   2017-03-24T20:31:35Z

    filter estimation on two columns of same table

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75223 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75223/testReport)** for PR 17415 at commit [`8930669`](https://github.com/apache/spark/commit/893066905b690c78a127eae58b908dff1dabf7cf).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    cc @sameeragarwal @cloud-fan @gatorsmile @wzhfy  After a few round of changes and commits, this PR should be in good shape.  If we can include in Spark 2.2, then it can help tpc-h queries.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109471156
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    Agreed.  We prefer over estimation to under estimation in order to avoid out-of-memory error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109097145
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    --- End diff --
    
    Could you add comments above this line like
    ```
            // Left < Right or Left <= Right
            // - no overlap:
            //      minRight           maxRight     minLeft       maxLeft
            // 0 ------+------------------+------------+-------------+------->
            // - complete overlap:
            //      minLeft            maxLeft      minRight      maxRight
            // 0 ------+------------------+------------+-------------+------->
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109099018
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    --- End diff --
    
    How about giving a complete list? Could you update the other functions too?
    
    // @param op a binary comparison operator, including =, <, <=, >, >=, <=>



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108308949
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -509,8 +524,131 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndv = BigDecimal(colStatLeft.distinctCount)
    +        var newNdv = (ndv * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    --- End diff --
    
    Good point.  Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109476536
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,221 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +          && (colStatLeft.distinctCount == colStatRight.distinctCount)
    +        )
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft)) && allNotNull,
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0 / 3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatRight.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            // Left < Right or Left <= Right
    +            //      minRight     <     minLeft
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinRight
    +            //
    +            //      maxRight     <     maxLeft
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxLeft
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            // Left > Right or Left >= Right
    +            //      minLeft     <      minRight
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinLeft
    +            //
    +            //      maxLeft     <      maxRight
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxRight
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            // Left = Right or Left <=> Right
    +            //      minLeft     <      minRight
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinLeft
    +            //
    +            //      minRight    <=     minLeft
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinRight
    +            //
    +            //      maxLeft     <      maxRight
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxRight
    +            //
    +            //      maxRight    <=     maxLeft
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxLeft
    +
    +
    +          if (minLeft < minRight) {
    +              newMinLeft = colStatRight.min
    +            } else {
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75411/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109297784
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    --- End diff --
    
    we can have a `val allNotNull = colStatLeft.nullCount == 0 && colStatRight.nullCount == 0`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75288 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75288/testReport)** for PR 17415 at commit [`9830a8f`](https://github.com/apache/spark/commit/9830a8ff3c9bb10e7d47b2b0dd9d6fa14d3fd2f7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109273334
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75335/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75186 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75186/testReport)** for PR 17415 at commit [`8930669`](https://github.com/apache/spark/commit/893066905b690c78a127eae58b908dff1dabf7cf).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109476460
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -491,7 +599,22 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
             sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
             rowCount = Some(expectedRowCount),
             attributeStats = expectedAttributeMap)
    -      assert(filter.stats(conf) == expectedStats)
    +
    +      val filterStats = filter.stats(conf)
    +      assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
    +      assert(filterStats.rowCount == expectedStats.rowCount)
    +      val rowCountValue = filterStats.rowCount.getOrElse(0)
    +      // check the output column stats if the row count is > 0.
    +      // When row count is 0, the output is set to empty.
    +      if (rowCountValue != 0) {
    +        // Need to check attributeStats one by one because we may have multiple output columns.
    +        // Due to update operation, the output columns may be in different order.
    +        expectedColStats.foreach { kv =>
    +          val filterColumnStat = filterStats.attributeStats.get(kv._1).get
    --- End diff --
    
    Good point.  fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108753830
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    --- End diff --
    
    I think it should be
    (minLeft > maxRight,  maxLeft < minRight)
    For no-overlap case, the condition should be "minLeft > maxRight" because we do not want any intersection point.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75472 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75472/testReport)** for PR 17415 at commit [`5a02705`](https://github.com/apache/spark/commit/5a02705bc3a3186dc5ec5855e78e26a8048df55e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109268076
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    --- End diff --
    
    The graphical  comments are helpful.  Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108584962
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    --- End diff --
    
    `(minLeft >= maxRight, maxLeft < minRight)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75477 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75477/testReport)** for PR 17415 at commit [`5a02705`](https://github.com/apache/spark/commit/5a02705bc3a3186dc5ec5855e78e26a8048df55e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109370506
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -491,7 +599,22 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
             sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
             rowCount = Some(expectedRowCount),
             attributeStats = expectedAttributeMap)
    -      assert(filter.stats(conf) == expectedStats)
    +
    +      val filterStats = filter.stats(conf)
    +      assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
    +      assert(filterStats.rowCount == expectedStats.rowCount)
    +      val rowCountValue = filterStats.rowCount.getOrElse(0)
    +      // check the output column stats if the row count is > 0.
    +      // When row count is 0, the output is set to empty.
    +      if (rowCountValue != 0) {
    +        // Need to check attributeStats one by one because we may have multiple output columns.
    +        // Due to update operation, the output columns may be in different order.
    +        expectedColStats.foreach { kv =>
    +          val filterColumnStat = filterStats.attributeStats.get(kv._1).get
    --- End diff --
    
    This only checks the `expectedColStats` is a sub-set of `filterStats.attributeStats`, shall we also check the size?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109103790
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            else newMinRight = colStatLeft.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +            else newMaxLeft = colStatRight.max
    --- End diff --
    
    Nit: code style fix:
    ```Scala
                if (minLeft < minRight) {
                  newMinLeft = colStatRight.min
                } else {
                  newMinRight = colStatLeft.min
                }
                if (maxLeft < maxRight) {
                  newMaxRight = colStatLeft.max
                } else {
                  newMaxLeft = colStatRight.max
                }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108307490
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -381,7 +461,22 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
             sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
             rowCount = Some(expectedRowCount),
             attributeStats = expectedAttributeMap)
    -      assert(filter.stats(conf) == expectedStats)
    +
    +      val filterStats = filter.stats(conf)
    +      assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
    +      assert(filterStats.rowCount == expectedStats.rowCount)
    +      val rowCountValue = filterStats.rowCount.getOrElse(0)
    +      // check the output column stats if the row count is > 0.
    +      // When row count is 0, the output is set to empty.
    +      if (rowCountValue != 0) {
    +        // Need to check attributeStats one by one because we may have multiple output columns.
    +        // Due to update operation, the output columns may be in different order.
    +        expectedColStats.foreach { kv =>
    +          val filterColumnStat = filterStats.attributeStats.get(kv._1).get
    +          assert(filterColumnStat == kv._2)
    +        }
    +      }
    +      // assert(filter.stats(conf) == expectedStats)
    --- End diff --
    
    My bad.  I should remove the line that has been commented out.  This line is replaced by the following code:
          if (rowCountValue != 0) {
            // Need to check attributeStats one by one because we may have multiple output columns.
            // Due to update operation, the output columns may be in different order.
            expectedColStats.foreach { kv =>
              val filterColumnStat = filterStats.attributeStats.get(kv._1).get
              assert(filterColumnStat == kv._2)
            }



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109476505
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,221 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +          && (colStatLeft.distinctCount == colStatRight.distinctCount)
    +        )
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft)) && allNotNull,
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108754614
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    --- End diff --
    
    I think it should be (maxLeft < minRight, minLeft > maxRight)
    For no-overlap case, the condition should be "maxLeft < minRight" because we do not want any intersection point.  It is a little bit complex.  This best way is to draw a diagram to show its relationship.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109298152
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    --- End diff --
    
    we need to update this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109340165
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    Good point.  We changed condition to:
    (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
              && (colStatLeft.distinctCount == colStatRight.distinctCount)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108585042
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    --- End diff --
    
    `(minLeft > maxRight, maxLeft <= minRight)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109323397
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109267675
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    --- End diff --
    
    Good catch.  fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109273331
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109096461
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    --- End diff --
    
    Nit: we can remove `ndvRight`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75449 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75449/testReport)** for PR 17415 at commit [`bf440db`](https://github.com/apache/spark/commit/bf440db0ee760de1e1cabe265a5129254a885a51).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75484 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75484/testReport)** for PR 17415 at commit [`3f3d30d`](https://github.com/apache/spark/commit/3f3d30d2f943f79cee8e5ab4ddf642f16104627a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75484 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75484/testReport)** for PR 17415 at commit [`3f3d30d`](https://github.com/apache/spark/commit/3f3d30d2f943f79cee8e5ab4ddf642f16104627a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75369 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75369/testReport)** for PR 17415 at commit [`70ac70c`](https://github.com/apache/spark/commit/70ac70cf0ab403e136d4114869174db171673364).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75438 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75438/testReport)** for PR 17415 at commit [`e2a1141`](https://github.com/apache/spark/commit/e2a1141b19c104c708e1648a841b2d2d1dfcebcc).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109381787
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    I doubt that the when this condition is true, it is a complete overlapping between two columns.
    
    The complete equality between the values of two columns also depends on the order. E.g., when left values are (1, 2, 3, 4), right values are (4, 3, 2, 1), the condition is true, but no values can pass the filter predicate `left_col = right_col`.
    
    Am I missing something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Thanks, Merging to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    It sounds like we have not supported a very common constant filter. Let me take a quick fix on that. : )


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109096448
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    --- End diff --
    
    Nit: we can remove `ndvLeft`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108582540
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -509,8 +524,131 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    --- End diff --
    
    I just revised the code to handle EqualNullSafe separately from EqualTo.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by wzhfy <gi...@git.apache.org>.
Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108601698
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109102176
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    --- End diff --
    
    ```
                // Left < Right or Left <= Right
                //      minRight     <     minLeft
                // 0 ------+******************+------->
                //              filtered      ^
                //                            |
                //                        newMinRight
                //
                //      maxRight     <     maxLeft
                // 0 ------+******************+------->
                //         ^    filtered
                //         |
                //     newMaxLeft
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75375/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108583642
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    --- End diff --
    
    `a given column` -> `two given columns`. Both two columns' `ColumnStat` are updated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109273326
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    --- End diff --
    
    fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108810108
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,8 +565,143 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: EqualTo =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            else newMinRight = colStatLeft.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +            else newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +        }
    +
    +        val newStatsLeft = colStatLeft.copy(distinctCount = newNdvLeft, min = newMinLeft,
    +          max = newMaxLeft)
    +        colStatsMap(attrLeft) = newStatsLeft
    +        val newStatsRight = colStatRight.copy(distinctCount = newNdvRight, min = newMinRight,
    +          max = newMaxRight)
    +        colStatsMap(attrRight) = newStatsRight
    +      }
    +    }
    +
    +    Some(percent.toDouble)
    +  }
    +
     }
     
    +
    --- End diff --
    
    nit: extra space.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75470/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75335 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75335/testReport)** for PR 17415 at commit [`7abed99`](https://github.com/apache/spark/commit/7abed99271064e27e86f7265a335b9bee0582d3a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75288/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75367/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75369/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108584825
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    --- End diff --
    
    (maxLeft <= minRight, minLeft > maxRight)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75367 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75367/testReport)** for PR 17415 at commit [`64bf43e`](https://github.com/apache/spark/commit/64bf43e562a3c257b847502eae651a8887eaddcf).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109323975
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && colStatLeft.nullCount == 0
    --- End diff --
    
    Estimation is always hard to be accurate. That is why user-provided hints are very useful for getting the right plan.
    
    I did a search. There are many different estimators. Uniform estimators, length estimator, digram estimator, minimum variance estimators, and histogram estimators. Is that possible we can consider the data distributions when deciding the selectivity?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108830896
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -450,6 +467,69 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
         }
       }
     
    +  test("cint = cint2") {
    +    validateEstimatedStats(
    +      Filter(EqualTo(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
    +      Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
    +        nullCount = 0, avgLen = 4, maxLen = 4),
    +        attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
    +          nullCount = 0, avgLen = 4, maxLen = 4)),
    +      expectedRowCount = 4)
    +  }
    +
    +  test("cint > cint2") {
    +    validateEstimatedStats(
    +      Filter(GreaterThan(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
    +      Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
    +        nullCount = 0, avgLen = 4, maxLen = 4),
    +        attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
    +          nullCount = 0, avgLen = 4, maxLen = 4)),
    +      expectedRowCount = 4)
    +  }
    +
    +  test("cint < cint2") {
    +    validateEstimatedStats(
    +      Filter(LessThan(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
    +      Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(1), max = Some(10),
    +        nullCount = 0, avgLen = 4, maxLen = 4),
    +        attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(16),
    +          nullCount = 0, avgLen = 4, maxLen = 4)),
    +      expectedRowCount = 4)
    +  }
    +
    +  test("cint = cint3") {
    +    // no records qualify due to no overlap
    +    validateEstimatedStats(
    +      Filter(EqualTo(attrInt, attrInt3), childStatsTestPlan(Seq(attrInt, attrInt3), 10L)),
    +      Seq(attrInt -> ColumnStat(distinctCount = 0, min = Some(1), max = Some(10),
    --- End diff --
    
    Once no overlap, is it still meaningful to keep `min`, `max`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109323394
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    --- End diff --
    
    See my comments below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75186/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108100730
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -381,7 +461,22 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
             sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
             rowCount = Some(expectedRowCount),
             attributeStats = expectedAttributeMap)
    -      assert(filter.stats(conf) == expectedStats)
    +
    +      val filterStats = filter.stats(conf)
    +      assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
    +      assert(filterStats.rowCount == expectedStats.rowCount)
    +      val rowCountValue = filterStats.rowCount.getOrElse(0)
    +      // check the output column stats if the row count is > 0.
    +      // When row count is 0, the output is set to empty.
    +      if (rowCountValue != 0) {
    +        // Need to check attributeStats one by one because we may have multiple output columns.
    +        // Due to update operation, the output columns may be in different order.
    +        expectedColStats.foreach { kv =>
    +          val filterColumnStat = filterStats.attributeStats.get(kv._1).get
    +          assert(filterColumnStat == kv._2)
    +        }
    +      }
    +      // assert(filter.stats(conf) == expectedStats)
    --- End diff --
    
    why comment it out?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75477 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75477/testReport)** for PR 17415 at commit [`5a02705`](https://github.com/apache/spark/commit/5a02705bc3a3186dc5ec5855e78e26a8048df55e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109099345
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            else newMinRight = colStatLeft.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +            else newMaxLeft = colStatRight.max
    --- End diff --
    
    Nit: the style needs a change for Spark coding style 
    ```Scala
                if (minLeft < minRight) {
                  newMinLeft = colStatRight.min
                } else {
                  newMinRight = colStatLeft.min
                }
                if (maxLeft < maxRight) {
                  newMaxRight = colStatLeft.max
                } else {
                  newMaxLeft = colStatRight.max
                }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75186 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75186/testReport)** for PR 17415 at commit [`8930669`](https://github.com/apache/spark/commit/893066905b690c78a127eae58b908dff1dabf7cf).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109289750
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    --- End diff --
    
    uh. I missed that. Please feel free to remove it. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108100193
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -509,8 +524,131 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    --- End diff --
    
    I think for `EqualTo`, there is no complete overlap.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109098218
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    --- End diff --
    
    ```
          // Left = Right or Left <=> Right
          // - no overlap:
          //      minLeft            maxLeft      minRight      maxRight
          // 0 ------+------------------+------------+-------------+------->
          //      minRight           maxRight     minLeft       maxLeft
          // 0 ------+------------------+------------+-------------+------->
          // - complete overlap:
          //      minLeft            maxLeft
          //      minRight           maxRight
          // 0 ------+------------------+------->
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109096766
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    --- End diff --
    
    Could you add comments like
    ```
            // no overlap: left < Right
            //      minRight           maxRight     minLeft       maxLeft
            // 0 ------+------------------+------------+-------------+------->
            // complete overlap: left < Right
            //      minLeft            maxLeft      minRight      maxRight
            // 0 ------+------------------+------------+-------------+------->
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    LGTM except some minor comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75284/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75470 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75470/testReport)** for PR 17415 at commit [`4f0b68f`](https://github.com/apache/spark/commit/4f0b68fbb271ad30f981d674f1192d886a5c5a03).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109289876
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    --- End diff --
    
    How about?
    ```
    (minRight == maxRight) && (minLeft == minRight) && (maxLeft == maxRight)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109298159
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && colStatLeft.nullCount == 0
    --- End diff --
    
    we need one more condition: `minLeft == minRight`, like @gatorsmile suggested https://github.com/apache/spark/pull/17415#discussion_r109289876


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75223/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109326608
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && colStatLeft.nullCount == 0
    --- End diff --
    
    Yes, estimation is always hard to be accurate.  We may consider supporting hints in the future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75288 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75288/testReport)** for PR 17415 at commit [`9830a8f`](https://github.com/apache/spark/commit/9830a8ff3c9bb10e7d47b2b0dd9d6fa14d3fd2f7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108582594
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,135 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    +    }
    --- End diff --
    
    Good catch.  Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75375 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75375/testReport)** for PR 17415 at commit [`9b98ff1`](https://github.com/apache/spark/commit/9b98ff1f7c8521e7d1277fd1f0c6e9a809a0d337).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75284 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75284/testReport)** for PR 17415 at commit [`d6a53ef`](https://github.com/apache/spark/commit/d6a53ef8ebbe470e83ef61e77e866c97bf1307a2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108307672
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala ---
    @@ -381,7 +461,22 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
             sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
             rowCount = Some(expectedRowCount),
             attributeStats = expectedAttributeMap)
    -      assert(filter.stats(conf) == expectedStats)
    +
    +      val filterStats = filter.stats(conf)
    +      assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
    +      assert(filterStats.rowCount == expectedStats.rowCount)
    +      val rowCountValue = filterStats.rowCount.getOrElse(0)
    +      // check the output column stats if the row count is > 0.
    +      // When row count is 0, the output is set to empty.
    +      if (rowCountValue != 0) {
    +        // Need to check attributeStats one by one because we may have multiple output columns.
    +        // Due to update operation, the output columns may be in different order.
    +        expectedColStats.foreach { kv =>
    +          val filterColumnStat = filterStats.attributeStats.get(kv._1).get
    +          assert(filterColumnStat == kv._2)
    +        }
    +      }
    +      // assert(filter.stats(conf) == expectedStats)
    --- End diff --
    
    My bad.  I should remove the line that has been commented out.  This line is replaced by the following code:
          if (rowCountValue != 0) {
            // Need to check attributeStats one by one because we may have multiple output columns.
            // Due to update operation, the output columns may be in different order.
            expectedColStats.foreach { kv =>
              val filterColumnStat = filterStats.attributeStats.get(kv._1).get
              assert(filterColumnStat == kv._2)
            } 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109323376
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,225 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight,
    +          maxLeft < minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight,
    +          maxLeft <= minRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight,
    +          minLeft > maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight,
    +          minLeft >= maxRight && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && colStatLeft.nullCount == 0
    --- End diff --
    
    You said "we need one more condition: minLeft == minRight".  Note that this condition is already included.
    
    @gatorsmile was suggesting "(minRight == maxRight) && (minLeft == minRight) && (maxLeft == maxRight)".  This implies all 4 values (minLeft, maxLeft, minRight, maxRight)  are equal.  This is not what I mean by complete overlap.  I initially defined "complete overlap" as "complete range overlap".  For example, we have a test case: test("cint = cint4").  If we use @gatorsmile's definition, then the case "cint = cint4" will become partial overlap with selectivity 0.33, which will under-estimate the selectivity.   In order to avoid out-of-memory error, I prefer over-estimating rather than under-estimating.   


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108751882
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    --- End diff --
    
    Good catch.  Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109404708
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    This is empirical. Without more statistics, it's really hard to do it mathematically.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109097352
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    --- End diff --
    
    ```
          // Left > Right or Left >= Right
          // - no overlap:
          //      minLeft            maxLeft      minRight      maxRight
          // 0 ------+------------------+------------+-------------+------->
          // - complete overlap:
          //      minRight           maxRight     minLeft       maxLeft
          // 0 ------+------------------+------------+-------------+------->
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109369718
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,221 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +          && (colStatLeft.distinctCount == colStatRight.distinctCount)
    +        )
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft)) && allNotNull,
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0 / 3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatRight.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            // Left < Right or Left <= Right
    +            //      minRight     <     minLeft
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinRight
    +            //
    +            //      maxRight     <     maxLeft
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxLeft
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            // Left > Right or Left >= Right
    +            //      minLeft     <      minRight
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinLeft
    +            //
    +            //      maxLeft     <      maxRight
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxRight
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            // Left = Right or Left <=> Right
    +            //      minLeft     <      minRight
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinLeft
    +            //
    +            //      minRight    <=     minLeft
    +            // --------+******************+------->
    +            //              filtered      ^
    +            //                            |
    +            //                        newMinRight
    +            //
    +            //      maxLeft     <      maxRight
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxRight
    +            //
    +            //      maxRight    <=     maxLeft
    +            // --------+******************+------->
    +            //         ^    filtered
    +            //         |
    +            //     newMaxLeft
    +
    +
    +          if (minLeft < minRight) {
    +              newMinLeft = colStatRight.min
    +            } else {
    --- End diff --
    
    The indention is wrong here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75375 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75375/testReport)** for PR 17415 at commit [`9b98ff1`](https://github.com/apache/spark/commit/9b98ff1f7c8521e7d1277fd1f0c6e9a809a0d337).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75367 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75367/testReport)** for PR 17415 at commit [`64bf43e`](https://github.com/apache/spark/commit/64bf43e562a3c257b847502eae651a8887eaddcf).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109102652
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    --- End diff --
    
    ```
                // Left > Right or Left >= Right
                //      minLeft     <      minRight
                // 0 ------+******************+------->
                //              filtered      ^
                //                            |
                //                        newMinLeft
                //
                //      maxLeft     <      maxRight
                // 0 ------+******************+------->
                //         ^    filtered
                //         |
                //     newMaxRight
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    ok to test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109293607
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    --- End diff --
    
    Good point.  fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    So far, for each op, we only have a single scenario. Do we need to improve the test case coverage for verifying all the scenarios?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75470 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75470/testReport)** for PR 17415 at commit [`4f0b68f`](https://github.com/apache/spark/commit/4f0b68fbb271ad30f981d674f1192d886a5c5a03).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75411 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75411/testReport)** for PR 17415 at commit [`fac7fce`](https://github.com/apache/spark/commit/fac7fce78ef025eb6fd734413e8d0ea5af11574d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109324280
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    Also compare the NDV?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75411 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75411/testReport)** for PR 17415 at commit [`fac7fce`](https://github.com/apache/spark/commit/fac7fce78ef025eb6fd734413e8d0ea5af11574d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109286505
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    --- End diff --
    
    doesn't the `complete overlap` here need to consider null?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109286468
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    --- End diff --
    
    the starting `0` looks confusing, the `max`, `min` values doesn't need to be positive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108100521
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -509,8 +524,131 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndv = BigDecimal(colStatLeft.distinctCount)
    +        var newNdv = (ndv * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    --- End diff --
    
    why the new `ndv` only look at `ndvLeft`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75484/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75449 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75449/testReport)** for PR 17415 at commit [`bf440db`](https://github.com/apache/spark/commit/bf440db0ee760de1e1cabe265a5129254a885a51).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75449/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    cc @sameeragarwal @cloud-fan @gatorsmile  This Jira is not on Spark 2.2 blocker list.  If time permits, we can include it in Spark 2.2.  If not, we can wait for a maintenance release.  Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108583109
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,135 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    --- End diff --
    
    The current code is written in such a way that we do not have too deep indentation.  Some engineers do not like deep indentation as they often put screen monitor vertically.
    Let's handle it when the need occurs.  I think, with good test case coverage, we will be able to catch anything we miss.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108314909
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,135 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: EqualTo =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            else newMinRight = colStatLeft.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +            else newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +        }
    +
    +        val newStatsLeft = colStatLeft.copy(distinctCount = newNdvLeft, min = newMinLeft,
    +          max = newMaxLeft, nullCount = 0)
    +        colStatsMap(attrLeft) = newStatsLeft
    +        val newStatsRight = colStatRight.copy(distinctCount = newNdvRight, min = newMinRight,
    +          max = newMaxRight, nullCount = 0)
    --- End diff --
    
    `nullCount ` might not be simply set to zero if we also support `EqualNullSafe `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109099995
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    --- End diff --
    
    Nit: `percent = 1.0 / 3.0`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108315514
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,135 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    --- End diff --
    
    Could we use white list here? It is also easy for us to see which data types are assumed to support in the implementation.
    
    I am afraid we might easily forget updating this if we support new data type in the future. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108314620
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,135 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    +    }
    --- End diff --
    
    We use `Equality` above, but we did not handle `EqualNullSafe`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109554814
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    OK. LGTM.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75369 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75369/testReport)** for PR 17415 at commit [`70ac70c`](https://github.com/apache/spark/commit/70ac70cf0ab403e136d4114869174db171673364).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75284 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75284/testReport)** for PR 17415 at commit [`d6a53ef`](https://github.com/apache/spark/commit/d6a53ef8ebbe470e83ef61e77e866c97bf1307a2).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75438/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109286524
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // 0 ------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // 0 ------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    --- End diff --
    
    I think `Left = Right` is different from the other 2 cases, even the range completely overlaps, the filter selectivity is not 100%.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108584830
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    --- End diff --
    
    `(maxLeft < minRight, minLeft >= maxRight)`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75472/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75438 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75438/testReport)** for PR 17415 at commit [`e2a1141`](https://github.com/apache/spark/commit/e2a1141b19c104c708e1648a841b2d2d1dfcebcc).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109104396
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,140 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft < minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft > maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft >= maxRight)
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0,
    +          ((minLeft == minRight) && (maxLeft == maxRight))
    +            && colStatLeft.nullCount == 0 && colStatRight.nullCount == 0
    +        )
    +    }
    +
    +    var percent = BigDecimal(1.0)
    +    if (noOverlap) {
    +      percent = 0.0
    +    } else if (completeOverlap) {
    +      percent = 1.0
    +    } else {
    +      // For partial overlap, we use an empirical value 1/3 as suggested by the book
    +      // "Database Systems, the complete book".
    +      percent = 1.0/3.0
    +
    +      if (update) {
    +        // Need to adjust new min/max after the filter condition is applied
    +
    +        val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvLeft < 1) newNdvLeft = 1
    +        val ndvRight = BigDecimal(colStatLeft.distinctCount)
    +        var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
    +        if (newNdvRight < 1) newNdvRight = 1
    +
    +        var newMaxLeft = colStatLeft.max
    +        var newMinLeft = colStatLeft.min
    +        var newMaxRight = colStatRight.max
    +        var newMinRight = colStatRight.min
    +
    +        op match {
    +          case _: LessThan | _: LessThanOrEqual =>
    +            // the left side should be less than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft > minRight) newMinRight = colStatLeft.min
    +            if (maxLeft > maxRight) newMaxLeft = colStatRight.max
    +
    +          case _: GreaterThan | _: GreaterThanOrEqual =>
    +            // the left side should be greater than the right side.
    +            // If not, we need to adjust it to narrow the range.
    +            if (minLeft < minRight) newMinLeft = colStatRight.min
    +            if (maxLeft < maxRight) newMaxRight = colStatLeft.max
    +
    +          case _: EqualTo | _: EqualNullSafe =>
    +            // need to set new min to the larger min value, and
    +            // set the new max to the smaller max value.
    --- End diff --
    
    ```
                // Left = Right or Left <=> Right
                //      minLeft     <      minRight
                // 0 ------+******************+------->
                //              filtered      ^
                //                            |
                //                        newMinLeft
                //
                //      minRight    <=     minLeft
                // 0 ------+******************+------->
                //              filtered      ^
                //                            |
                //                        newMinRight
                //
                //      maxLeft     <      maxRight
                // 0 ------+******************+------->
                //         ^    filtered
                //         |
                //     newMaxRight
                //
                //      maxRight    <=     maxLeft
                // 0 ------+******************+------->
                //         ^    filtered
                //         |
                //     newMaxLeft
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/17415


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108246637
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -509,8 +524,131 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    --- End diff --
    
    The left side has a range [minLeft, maxLeft].  The right side has a range [minRight, maxRight].  If the range overlaps, then we assume that they have a complete overlap.  Without detailed/advanced statistics, this is the best estimate we can get.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r109368811
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -550,6 +565,221 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      // Left < Right or Left <= Right
    +      // - no overlap:
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      case _: LessThan =>
    +        (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
    +      case _: LessThanOrEqual =>
    +        (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
    +
    +      // Left > Right or Left >= Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap: (If null values exists, we set it to partial overlap.)
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
    +
    +      // Left = Right or Left <=> Right
    +      // - no overlap:
    +      //      minLeft            maxLeft      minRight      maxRight
    +      // --------+------------------+------------+-------------+------->
    +      //      minRight           maxRight     minLeft       maxLeft
    +      // --------+------------------+------------+-------------+------->
    +      // - complete overlap:
    +      //      minLeft            maxLeft
    +      //      minRight           maxRight
    +      // --------+------------------+------->
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    +          && (colStatLeft.distinctCount == colStatRight.distinctCount)
    +        )
    +      case _: EqualNullSafe =>
    +        // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
    +        // If null values exists, we set it to partial overlap.
    +        (((maxLeft < minRight) || (maxRight < minLeft)) && allNotNull,
    +          (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
    --- End diff --
    
    `&& (colStatLeft.distinctCount == colStatRight.distinctCount)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #17415: [SPARK-19408][SQL] filter estimation on two colum...

Posted by ron8hu <gi...@git.apache.org>.
Github user ron8hu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17415#discussion_r108752975
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala ---
    @@ -515,8 +530,138 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
         Some(percent.toDouble)
       }
     
    +  /**
    +   * Returns a percentage of rows meeting a binary comparison expression containing two columns.
    +   * In SQL queries, we also see predicate expressions involving two columns
    +   * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
    +   * Note that, if column-1 and column-2 belong to different tables, then it is a join
    +   * operator's work, NOT a filter operator's work.
    +   *
    +   * @param op a binary comparison operator such as =, <, <=, >, >=
    +   * @param attrLeft the left Attribute (or a column)
    +   * @param attrRight the right Attribute (or a column)
    +   * @param update a boolean flag to specify if we need to update ColumnStat of a given column
    +   *               for subsequent conditions
    +   * @return an optional double value to show the percentage of rows meeting a given condition
    +   */
    +  def evaluateBinaryForTwoColumns(
    +      op: BinaryComparison,
    +      attrLeft: Attribute,
    +      attrRight: Attribute,
    +      update: Boolean): Option[Double] = {
    +
    +    if (!colStatsMap.contains(attrLeft)) {
    +      logDebug("[CBO] No statistics for " + attrLeft)
    +      return None
    +    }
    +    if (!colStatsMap.contains(attrRight)) {
    +      logDebug("[CBO] No statistics for " + attrRight)
    +      return None
    +    }
    +
    +    attrLeft.dataType match {
    +      case StringType | BinaryType =>
    +        // TODO: It is difficult to support other binary comparisons for String/Binary
    +        // type without min/max and advanced statistics like histogram.
    +        logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
    +        return None
    +      case _ =>
    +    }
    +
    +    val colStatLeft = colStatsMap(attrLeft)
    +    val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxLeft = BigDecimal(statsRangeLeft.max)
    +    val minLeft = BigDecimal(statsRangeLeft.min)
    +    val ndvLeft = BigDecimal(colStatLeft.distinctCount)
    +
    +    val colStatRight = colStatsMap(attrRight)
    +    val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
    +      .asInstanceOf[NumericRange]
    +    val maxRight = BigDecimal(statsRangeRight.max)
    +    val minRight = BigDecimal(statsRangeRight.min)
    +    val ndvRight = BigDecimal(colStatRight.distinctCount)
    +
    +    // determine the overlapping degree between predicate range and column's range
    +    val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
    +      case _: EqualTo =>
    +        ((maxLeft < minRight) || (maxRight < minLeft),
    +          (minLeft == minRight) && (maxLeft == maxRight))
    +      case _: LessThan =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: LessThanOrEqual =>
    +        (minLeft >= maxRight, maxLeft <= minRight)
    +      case _: GreaterThan =>
    +        (maxLeft <= minRight, minLeft >= maxRight)
    +      case _: GreaterThanOrEqual =>
    +        (maxLeft < minRight, minLeft > maxRight)
    --- End diff --
    
    Good catch.  Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75477/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75223 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75223/testReport)** for PR 17415 at commit [`8930669`](https://github.com/apache/spark/commit/893066905b690c78a127eae58b908dff1dabf7cf).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    **[Test build #75335 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75335/testReport)** for PR 17415 at commit [`7abed99`](https://github.com/apache/spark/commit/7abed99271064e27e86f7265a335b9bee0582d3a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #17415: [SPARK-19408][SQL] filter estimation on two columns of s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/17415
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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