You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by wangyum <gi...@git.apache.org> on 2018/04/28 11:26:44 UTC

[GitHub] spark pull request #21189: [SPARK-24117][SQL] Unified the getSizePerRow

GitHub user wangyum opened a pull request:

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

    [SPARK-24117][SQL] Unified the getSizePerRow

    ## What changes were proposed in this pull request?
    
    This pr unified the `getSizePerRow` because `getSizePerRow` is used in many places. For example:
    
    1. [LocalRelation.scala#L80](https://github.com/wangyum/spark/blob/f70f46d1e5bc503e9071707d837df618b7696d32/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala#L80)
    2. [SizeInBytesOnlyStatsPlanVisitor.scala#L36](https://github.com/apache/spark/blob/76b8b840ddc951ee6203f9cccd2c2b9671c1b5e8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala#L36)
    
    
    ## How was this patch tested?
    Exist tests


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

    $ git pull https://github.com/wangyum/spark SPARK-24117

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

    https://github.com/apache/spark/pull/21189.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 #21189
    
----
commit cd415381386f0ac5c29cd6dab57ceafc86e96adf
Author: Yuming Wang <yu...@...>
Date:   2018-04-28T11:10:33Z

    Unified the getSizePerRow

----


---

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


[GitHub] spark pull request #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185652723
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala ---
    @@ -220,11 +220,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter {
     
         sink.addBatch(0, 1 to 3)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 12)
    +    assert(plan.stats.sizeInBytes === 36)
     
         sink.addBatch(1, 4 to 6)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 24)
    +    assert(plan.stats.sizeInBytes === 72)
    --- End diff --
    
    SGTM then


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

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


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185167132
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala ---
    @@ -220,11 +220,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter {
     
         sink.addBatch(0, 1 to 3)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 12)
    +    assert(plan.stats.sizeInBytes === 36)
     
         sink.addBatch(1, 4 to 6)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 24)
    +    assert(plan.stats.sizeInBytes === 72)
    --- End diff --
    
    It shouldn't impact anything, but abstractly it seems strange that this unification would cause the stats to change? What are we doing differently to cause this, and how confident are we this won't happen to production sinks?


---

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


[GitHub] spark pull request #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185042726
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala ---
    @@ -220,11 +220,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter {
     
         sink.addBatch(0, 1 to 3)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 12)
    +    assert(plan.stats.sizeInBytes === 36)
     
         sink.addBatch(1, 4 to 6)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 24)
    +    assert(plan.stats.sizeInBytes === 72)
    --- End diff --
    
    `MemorySinkV2 ` is mainly for testing. I think the stats changes will not impact anything, right? @tdas @jose-torres 


---

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


[GitHub] spark pull request #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185041473
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala ---
    @@ -178,7 +179,7 @@ class MemoryDataWriter(partition: Int, outputMode: OutputMode)
      * Used to query the data that has been written into a [[MemorySinkV2]].
      */
     case class MemoryPlanV2(sink: MemorySinkV2, override val output: Seq[Attribute]) extends LeafNode {
    -  private val sizePerRow = output.map(_.dataType.defaultSize).sum
    --- End diff --
    
    @tdas @jose-torres Is that possible this can be zero?


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89955/
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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/21189#discussion_r185515489
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala ---
    @@ -220,11 +220,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter {
     
         sink.addBatch(0, 1 to 3)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 12)
    +    assert(plan.stats.sizeInBytes === 36)
     
         sink.addBatch(1, 4 to 6)
         plan.invalidateStatsCache()
    -    assert(plan.stats.sizeInBytes === 24)
    +    assert(plan.stats.sizeInBytes === 72)
    --- End diff --
    
    It seems we forgot to count the row object overhead (8 bytes) before in memory stream.


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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/3037/
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185043356
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala ---
    @@ -33,8 +33,8 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] {
       private def visitUnaryNode(p: UnaryNode): Statistics = {
         // There should be some overhead in Row object, the size should not be zero when there is
         // no columns, this help to prevent divide-by-zero error.
    -    val childRowSize = p.child.output.map(_.dataType.defaultSize).sum + 8
    -    val outputRowSize = p.output.map(_.dataType.defaultSize).sum + 8
    +    val childRowSize = EstimationUtils.getSizePerRow(p.child.output)
    +    val outputRowSize = EstimationUtils.getSizePerRow(p.output)
    --- End diff --
    
    cc @juliuszsompolski @cloud-fan 


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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/2738/
    Test PASSed.


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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/21189#discussion_r185514873
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala ---
    @@ -33,8 +33,8 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] {
       private def visitUnaryNode(p: UnaryNode): Statistics = {
         // There should be some overhead in Row object, the size should not be zero when there is
         // no columns, this help to prevent divide-by-zero error.
    -    val childRowSize = p.child.output.map(_.dataType.defaultSize).sum + 8
    -    val outputRowSize = p.output.map(_.dataType.defaultSize).sum + 8
    +    val childRowSize = EstimationUtils.getSizePerRow(p.child.output)
    --- End diff --
    
    I think we still need to keep the `+ 8`


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    thanks, merging to master!


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    **[Test build #89955 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89955/testReport)** for PR 21189 at commit [`cd41538`](https://github.com/apache/spark/commit/cd415381386f0ac5c29cd6dab57ceafc86e96adf).
     * 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 pull request #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

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


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

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


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

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


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

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


---

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


[GitHub] spark issue #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    **[Test build #90365 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90365/testReport)** for PR 21189 at commit [`f72084e`](https://github.com/apache/spark/commit/f72084e9edfbdaf6334b13c54f1534eca5e88806).
     * 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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189
  
    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 #21189: [SPARK-24117][SQL] Unified the getSizePerRow

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

    https://github.com/apache/spark/pull/21189#discussion_r185167182
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala ---
    @@ -178,7 +179,7 @@ class MemoryDataWriter(partition: Int, outputMode: OutputMode)
      * Used to query the data that has been written into a [[MemorySinkV2]].
      */
     case class MemoryPlanV2(sink: MemorySinkV2, override val output: Seq[Attribute]) extends LeafNode {
    -  private val sizePerRow = output.map(_.dataType.defaultSize).sum
    --- End diff --
    
    I wouldn't think it's possible.


---

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