You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by maropu <gi...@git.apache.org> on 2018/06/29 06:58:32 UTC

[GitHub] spark pull request #21668: [SPARK-24690][SQL] Add a new config to control pl...

GitHub user maropu opened a pull request:

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

    [SPARK-24690][SQL] Add a new config to control plan stats computation in LogicalRelation

    ## What changes were proposed in this pull request?
    This pr proposes to propose a new separate config so that `LogicalRelation` can use `rowCount` to compute data statistics in logical plans. In the master, we currently cannot enable `StarSchemaDetection.reorderStarJoins` because we need to turn off CBO to enable it but `StarSchemaDetection` internally references the `rowCount` that is used in LogicalRelation if CBO disabled.
    
    ## How was this patch tested?
    Added tests in `DataFrameJoinSuite`.

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

    $ git pull https://github.com/maropu/spark PlanStatsConf

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

    https://github.com/apache/spark/pull/21668.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 #21668
    
----
commit f0db73bcad7741ec06560a1b6bb50bb49483b954
Author: Takeshi Yamamuro <ya...@...>
Date:   2018-06-29T06:22:16Z

    Fix

----


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    @cloud-fan ping


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    sounds reasonable, also cc @wzhfy @maryannxue 


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    yea this is a real problem, but I feel a better solution is to integrate the StarSchemaDetection into CBO. How hard will it be?


---

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


[GitHub] spark pull request #21668: [SPARK-24690][SQL] Add a new config to control pl...

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

    https://github.com/apache/spark/pull/21668#discussion_r199552405
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -375,16 +375,16 @@ case class CatalogStatistics(
        * Convert [[CatalogStatistics]] to [[Statistics]], and match column stats to attributes based
        * on column names.
        */
    -  def toPlanStats(planOutput: Seq[Attribute], cboEnabled: Boolean): Statistics = {
    -    if (cboEnabled && rowCount.isDefined) {
    +  def toPlanStats(planOutput: Seq[Attribute], planStatsEnabled: Boolean): Statistics = {
    +    if (planStatsEnabled && rowCount.isDefined) {
           val attrStats = AttributeMap(planOutput
             .flatMap(a => colStats.get(a.name).map(a -> _.toPlanStat(a.name, a.dataType))))
           // Estimate size as number of rows * row size.
           val size = EstimationUtils.getOutputSize(planOutput, rowCount.get, attrStats)
           Statistics(sizeInBytes = size, rowCount = rowCount, attributeStats = attrStats)
         } else {
    -      // When CBO is disabled or the table doesn't have other statistics, we apply the size-only
    -      // estimation strategy and only propagate sizeInBytes in statistics.
    +      // When plan statistics are disabled or the table doesn't have other statistics,
    +      // we apply the size-only estimation strategy and only propagate sizeInBytes in statistics.
           Statistics(sizeInBytes = sizeInBytes)
    --- End diff --
    
    cc @wzhfy 


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark pull request #21668: [SPARK-24690][SQL] Add a new config to control pl...

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

    https://github.com/apache/spark/pull/21668#discussion_r199552346
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -375,16 +375,16 @@ case class CatalogStatistics(
        * Convert [[CatalogStatistics]] to [[Statistics]], and match column stats to attributes based
        * on column names.
        */
    -  def toPlanStats(planOutput: Seq[Attribute], cboEnabled: Boolean): Statistics = {
    -    if (cboEnabled && rowCount.isDefined) {
    +  def toPlanStats(planOutput: Seq[Attribute], planStatsEnabled: Boolean): Statistics = {
    +    if (planStatsEnabled && rowCount.isDefined) {
           val attrStats = AttributeMap(planOutput
             .flatMap(a => colStats.get(a.name).map(a -> _.toPlanStat(a.name, a.dataType))))
           // Estimate size as number of rows * row size.
           val size = EstimationUtils.getOutputSize(planOutput, rowCount.get, attrStats)
           Statistics(sizeInBytes = size, rowCount = rowCount, attributeStats = attrStats)
         } else {
    -      // When CBO is disabled or the table doesn't have other statistics, we apply the size-only
    -      // estimation strategy and only propagate sizeInBytes in statistics.
    +      // When plan statistics are disabled or the table doesn't have other statistics,
    +      // we apply the size-only estimation strategy and only propagate sizeInBytes in statistics.
           Statistics(sizeInBytes = sizeInBytes)
    --- End diff --
    
    I think we should respect rowCount, if it is available, no matter whether CBO is on or off. 


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    This comes from #20345.


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    **[Test build #93382 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93382/testReport)** for PR 21668 at commit [`0b1f751`](https://github.com/apache/spark/commit/0b1f7513fc99463f05d1de7585d4cd9f69f47ddf).


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark pull request #21668: [SPARK-24690][SQL] Add a new config to control pl...

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

    https://github.com/apache/spark/pull/21668#discussion_r199682495
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -375,16 +375,16 @@ case class CatalogStatistics(
        * Convert [[CatalogStatistics]] to [[Statistics]], and match column stats to attributes based
        * on column names.
        */
    -  def toPlanStats(planOutput: Seq[Attribute], cboEnabled: Boolean): Statistics = {
    -    if (cboEnabled && rowCount.isDefined) {
    +  def toPlanStats(planOutput: Seq[Attribute], planStatsEnabled: Boolean): Statistics = {
    +    if (planStatsEnabled && rowCount.isDefined) {
           val attrStats = AttributeMap(planOutput
             .flatMap(a => colStats.get(a.name).map(a -> _.toPlanStat(a.name, a.dataType))))
           // Estimate size as number of rows * row size.
           val size = EstimationUtils.getOutputSize(planOutput, rowCount.get, attrStats)
           Statistics(sizeInBytes = size, rowCount = rowCount, attributeStats = attrStats)
         } else {
    -      // When CBO is disabled or the table doesn't have other statistics, we apply the size-only
    -      // estimation strategy and only propagate sizeInBytes in statistics.
    +      // When plan statistics are disabled or the table doesn't have other statistics,
    +      // we apply the size-only estimation strategy and only propagate sizeInBytes in statistics.
           Statistics(sizeInBytes = sizeInBytes)
    --- End diff --
    
    yea, I see. We might do so.


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    ping


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1199/
    Test PASSed.


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    yea, ok. I'll reconsider this again. Thanks!


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    @cloud-fan If no problem, could you check #20345 and merge it first? Based on that, I'd like to start refactoring for the approach. 


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

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


---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    One of refactoring ideas is to inject the functionality of `ReorderJoin`(=`StarSchemaDetection`)
    into `CostBasedJoinReorder`;
    
    In [the batch rule `Join Reorder` (`Once` strategy)](https://github.com/apache/spark/blob/7c08eb6d61d55ce45229f3302e6d463e7669183d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L139), if `spark.sql.cbo.starSchemaDetection` enabled (false by default), the rule applies star schema detection first. If a fact table found, [dimension tables are reordered](https://github.com/apache/spark/blob/7c08eb6d61d55ce45229f3302e6d463e7669183d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala#L342) by the cost-based algorithm. If `spark.sql.cbo.starSchemaDetection` disabled, the rule just uses `CostBasedJoinReorder`.
    
    Currently, we have `ReorderJoin`(=`StarSchemaDetection`) in the batch rule with `fixedPoint` strategy,
    so, I thnk that, if we could remove this rule from there, we would skip unnecessary checks caused by `ReorderJoin` per rule iteration.
    
    @cloud-fan WDYT?



---

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


[GitHub] spark issue #21668: [SPARK-24690][SQL] Add a new config to control plan stat...

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

    https://github.com/apache/spark/pull/21668
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/572/
    Test PASSed.


---

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