You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by YanTangZhai <gi...@git.apache.org> on 2014/09/16 15:26:27 UTC

[GitHub] spark pull request: [SPARK-3545] Put HadoopRDD.getPartitions forwa...

GitHub user YanTangZhai opened a pull request:

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

    [SPARK-3545] Put HadoopRDD.getPartitions forward and put TaskScheduler.start back to reduce DAGScheduler.JobSubmitted processing time and shorten cluster resources occupation period

    We have two problems:
    (1) HadoopRDD.getPartitions is lazyied to process in DAGScheduler.JobSubmitted. If inputdir is large, getPartitions may spend much time.
    For example, in our cluster, it needs from 0.029s to 766.699s. If one JobSubmitted event is processing, others should wait. Thus, we
    want to put HadoopRDD.getPartitions forward to reduce DAGScheduler.JobSubmitted processing time. Then other JobSubmitted event don't
    need to wait much time. HadoopRDD object could get its partitons when it is instantiated.
    (2) When SparkContext object is instantiated, TaskScheduler is started and some resources are allocated from cluster. However, these
    resources may be not used for the moment. For example, DAGScheduler.JobSubmitted is processing and so on. These resources are wasted in
    this period. Thus, we want to put TaskScheduler.start back to shorten cluster resources occupation period specially for busy cluster.
    TaskScheduler could be started just before running stages.
    We could analyse and compare the execution time before and after optimization.
    TaskScheduler.start execution time: [time1__]
    DAGScheduler.JobSubmitted (excluding HadoopRDD.getPartitions or TaskScheduler.start) execution time: [time2_]
    HadoopRDD.getPartitions execution time: [time3___]
    Stages execution time: [time4_____]
    (1) The app has only one job
    (a)
    The execution time of the job before optimization is [time1__][time2_][time3___][time4_____].
    The execution time of the job after optimization is....[time3___][time2_][time1__][time4_____].
    (b)
    The cluster resources occupation period before optimization is [time2_][time3___][time4_____].
    The cluster resources occupation period after optimization is....[time4_____].
    In summary, if the app has only one job, the total execution time is same before and after optimization while the cluster resources
    occupation period after optimization is less than before.
    (2) The app has 4 jobs
    (a) Before optimization,
    job1 execution time is [time2_][time3___][time4_____],
    job2 execution time is [time2__________][time3___][time4_____],
    job3 execution time is................................[time2____][time3___][time4_____],
    job4 execution time is................................[time2______________][time3___][time4_____].
    After optimization, 
    job1 execution time is [time3___][time2_][time1__][time4_____],
    job2 execution time is [time3___][time2__________][time4_____],
    job3 execution time is................................[time3___][time2_][time4_____],
    job4 execution time is................................[time3___][time2__][time4_____].
    In summary, if the app has multiple jobs, average execution time after optimization is less than before and the cluster resources
    occupation period after optimization is less than before.

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

    $ git pull https://github.com/YanTangZhai/spark SPARK-3545

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

    https://github.com/apache/spark/pull/2409.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 #2409
    
----
commit cdef539abc5d2d42d4661373939bdd52ca8ee8e6
Author: YanTangZhai <ha...@tencent.com>
Date:   2014-08-06T13:07:08Z

    Merge pull request #1 from apache/master
    
    update

commit cbcba66ad77b96720e58f9d893e87ae5f13b2a95
Author: YanTangZhai <ha...@tencent.com>
Date:   2014-08-20T13:14:08Z

    Merge pull request #3 from apache/master
    
    Update

commit 8a0010691b669495b4c327cf83124cabb7da1405
Author: YanTangZhai <ha...@tencent.com>
Date:   2014-09-12T06:54:58Z

    Merge pull request #6 from apache/master
    
    Update

commit 03b62b043ab7fd39300677df61c3d93bb9beb9e3
Author: YanTangZhai <ha...@tencent.com>
Date:   2014-09-16T12:03:22Z

    Merge pull request #7 from apache/master
    
    Update

commit b88df438033eecbdbe8cad37b2bd4ad3620de6e2
Author: yantangzhai <ty...@163.com>
Date:   2014-09-16T13:22:12Z

    [SPARK-3545] Put HadoopRDD.getPartitions forward and put TaskScheduler.start back to reduce DAGScheduler.JobSubmitted processing time and shorten cluster resources occupation period

----


---
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-3545] Put HadoopRDD.getPartitions forwa...

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

    https://github.com/apache/spark/pull/2409#issuecomment-75142292
  
    Hi @YanTangZhai seems that there is some duplicate work between this and your other patch #3810. Can we close this issue?


---
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-3545] Put HadoopRDD.getPartitions forwa...

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

    https://github.com/apache/spark/pull/2409#issuecomment-55741743
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20385/consoleFull) for   PR 2409 at commit [`b88df43`](https://github.com/apache/spark/commit/b88df438033eecbdbe8cad37b2bd4ad3620de6e2).
     * This patch merges cleanly.


---
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-3545] Put HadoopRDD.getPartitions forwa...

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

    https://github.com/apache/spark/pull/2409#issuecomment-68009895
  
    It looks like there are two sort-of-unrelated changes in this PR:
    
    - Don't perform the expensive `HadoopRDD.getPartitions()` call inside of DAGScheduler, since this blocks the scheduling of other jobs / tasks.
    - Don't start the task scheduler until the first task is submitted.
    
    I think that we should handle these in separate JIRAs / PRs.


---
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-3545] Put HadoopRDD.getPartitions forwa...

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

    https://github.com/apache/spark/pull/2409#issuecomment-55749156
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20385/consoleFull) for   PR 2409 at commit [`b88df43`](https://github.com/apache/spark/commit/b88df438033eecbdbe8cad37b2bd4ad3620de6e2).
     * This patch **fails** 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: [SPARK-3545] Put HadoopRDD.getPartitions forwa...

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

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


---
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-3545] Put HadoopRDD.getPartitions forwa...

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

    https://github.com/apache/spark/pull/2409#issuecomment-68021964
  
    @JoshRosen Thanks. I will divide this JIRA/PR into two JIRAs/PRs.


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