You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by JoshRosen <gi...@git.apache.org> on 2015/10/26 03:20:08 UTC

[GitHub] spark pull request: [SPARK-11307] Reduce memory consumption of Out...

GitHub user JoshRosen opened a pull request:

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

    [SPARK-11307] Reduce memory consumption of OutputCommitCoordinator

    OutputCommitCoordinator uses a map in a place where an array would suffice, increasing its memory consumption for result stages with millions of tasks.
    
    This patch replaces that map with an array. The only tricky part of this is reasoning about the range of possible array indexes in order to make sure that we never index out of bounds.

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

    $ git pull https://github.com/JoshRosen/spark SPARK-11307

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

    https://github.com/apache/spark/pull/9274.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 #9274
    
----
commit 9dc210e0b37bef7c08806f9292b4b66fae5a142e
Author: Josh Rosen <jo...@databricks.com>
Date:   2015-10-26T02:18:43Z

    [SPARK-11307] Reduce memory consumption of OutputCommitCoordinator.

----


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#discussion_r43323114
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -949,7 +949,13 @@ class DAGScheduler(
         // serializable. If tasks are not serializable, a SparkListenerStageCompleted event
         // will be posted, which should always come after a corresponding SparkListenerStageSubmitted
         // event.
    -    outputCommitCoordinator.stageStart(stage.id)
    +    stage match {
    +      case s: ShuffleMapStage =>
    +        outputCommitCoordinator.stageStart(stage = s.id, maxPartitionId = s.numPartitions - 1)
    --- End diff --
    
    Yeah, it should be irrelevant for ShuffleMapStages. I was just being overly-conservative 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 pull request: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153918821
  
    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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151023940
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44327/
    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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#discussion_r42955382
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -949,7 +949,13 @@ class DAGScheduler(
         // serializable. If tasks are not serializable, a SparkListenerStageCompleted event
         // will be posted, which should always come after a corresponding SparkListenerStageSubmitted
         // event.
    -    outputCommitCoordinator.stageStart(stage.id)
    +    stage match {
    +      case s: ShuffleMapStage =>
    +        outputCommitCoordinator.stageStart(stage = s.id, maxPartitionId = s.numPartitions - 1)
    +      case s: ResultStage =>
    +        outputCommitCoordinator.stageStart(
    +          stage = s.id, maxPartitionId = s.rdd.partitions.length - 1)
    --- End diff --
    
    This result stage case is trickier: for the cases where the OutputCommitCoordinator actually gets invoked, I think it's generally the case that all partitions are being computed, but I guess it's hypothetically possible that a result stage could write results for only one of the RDD's partitions. In this case, I think the partition ids can be larger than `numPartitions - 1` (see the constructor of ResultStage to see 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 pull request: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#discussion_r42955341
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -949,7 +949,13 @@ class DAGScheduler(
         // serializable. If tasks are not serializable, a SparkListenerStageCompleted event
         // will be posted, which should always come after a corresponding SparkListenerStageSubmitted
         // event.
    -    outputCommitCoordinator.stageStart(stage.id)
    +    stage match {
    +      case s: ShuffleMapStage =>
    +        outputCommitCoordinator.stageStart(stage = s.id, maxPartitionId = s.numPartitions - 1)
    --- End diff --
    
    A shuffle map stage's maximum partition id is determined by the number of partitions in the RDD being computed.


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151023937
  
    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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153896037
  
    **[Test build #45056 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45056/consoleFull)** for PR 9274 at commit [`5085aa8`](https://github.com/apache/spark/commit/5085aa8688fc051d26bdbad3afc3d79de6645e3d).


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153894131
  
    Merged build started.


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153918823
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45056/
    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: [SPARK-11307] Reduce memory consumption of Out...

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

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


---
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: [SPARK-11307] Reduce memory consumption of Out...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9274#issuecomment-153923346
  
    Merged into master, 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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151005047
  
    **[Test build #44327 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44327/consoleFull)** for PR 9274 at commit [`9dc210e`](https://github.com/apache/spark/commit/9dc210e0b37bef7c08806f9292b4b66fae5a142e).


---
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: [SPARK-11307] Reduce memory consumption of Out...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/9274#issuecomment-151004298
  
    /cc @kayousterhout @markhamstra, this seems like a potentially easy win for reducing driver memory consumption when performing a write that outputs millions of partitions. This isn't necessarily a huge amount of memory savings, but it's a substantial reduction in the number of map entry objects created, which could have GC benefits.


---
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: [SPARK-11307] Reduce memory consumption of Out...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the pull request:

    https://github.com/apache/spark/pull/9274#issuecomment-151626530
  
    one small question, overall lgtm.  but I'm not very familiar w/ the speculative execution code so would appreciate an expert opinion.


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153918731
  
    **[Test build #45056 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45056/consoleFull)** for PR 9274 at commit [`5085aa8`](https://github.com/apache/spark/commit/5085aa8688fc051d26bdbad3afc3d79de6645e3d).
     * 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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151003850
  
    Merged build started.


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151003777
  
     Merged build triggered.


---
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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-151023885
  
    **[Test build #44327 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44327/consoleFull)** for PR 9274 at commit [`9dc210e`](https://github.com/apache/spark/commit/9dc210e0b37bef7c08806f9292b4b66fae5a142e).
     * 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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#discussion_r43177740
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -949,7 +949,13 @@ class DAGScheduler(
         // serializable. If tasks are not serializable, a SparkListenerStageCompleted event
         // will be posted, which should always come after a corresponding SparkListenerStageSubmitted
         // event.
    -    outputCommitCoordinator.stageStart(stage.id)
    +    stage match {
    +      case s: ShuffleMapStage =>
    +        outputCommitCoordinator.stageStart(stage = s.id, maxPartitionId = s.numPartitions - 1)
    --- End diff --
    
    as I was reviewing this, I was wondering if a `ShuffleMapStage` could have a different maximum partitionId if it was from a skipped stage.  I'm now convinced it cannot, but it might be a bit clearer if we change the [constructor](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala#L39) to not even take a `numTasks` argument, since it should always be `rdd.partitions.length`?  Not necessary for this change, but just a thought while you are touching this.
    
    Also -- isn't the output commit coordinator irrelevant for `ShuffleMapStage`s anyway?  If not, than I think there might be another bug there for skipped stages.  Since it indexes by stageId, you can have two different stages, that really represent the exact same shuffle, so you could have two different tasks authorized to commit that are handling the same stage.  (Which wouldn't be a problem introduced by this change, but I just thought it was worth mentioning.)


---
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: [SPARK-11307] Reduce memory consumption of Out...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/9274#issuecomment-153843232
  
    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: [SPARK-11307] Reduce memory consumption of Out...

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

    https://github.com/apache/spark/pull/9274#issuecomment-153894095
  
     Merged build triggered.


---
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