You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by drcrallen <gi...@git.apache.org> on 2016/06/16 18:14:33 UTC

[GitHub] spark pull request #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

GitHub user drcrallen opened a pull request:

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

    [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedulerBackend offer consideration

    The offer acceptance workflow is a little hard to follow and not very extensible for future considerations for offers. This is a patch that makes the workflow a little more explicit in its handling of offer resources.

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

    $ git pull https://github.com/metamx/spark SPARK15992

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

    https://github.com/apache/spark/pull/13715.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 #13715
    
----

----


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    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 issue #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

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

    https://github.com/apache/spark/pull/13715#discussion_r74952999
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala ---
    @@ -382,59 +382,97 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
           for (offer <- offers) {
             val slaveId = offer.getSlaveId.getValue
             val offerId = offer.getId.getValue
    -        val resources = remainingResources(offerId)
    -
    -        if (canLaunchTask(slaveId, resources)) {
    -          // Create a task
    -          launchTasks = true
    -          val taskId = newMesosTaskId()
    -          val offerCPUs = getResource(resources, "cpus").toInt
    -
    -          val taskCPUs = executorCores(offerCPUs)
    -          val taskMemory = executorMemory(sc)
    -
    -          slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
    -
    -          val (afterCPUResources, cpuResourcesToUse) =
    -            partitionResources(resources, "cpus", taskCPUs)
    -          val (resourcesLeft, memResourcesToUse) =
    -            partitionResources(afterCPUResources.asJava, "mem", taskMemory)
    -
    -          val taskBuilder = MesosTaskInfo.newBuilder()
    -            .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
    -            .setSlaveId(offer.getSlaveId)
    -            .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
    -            .setName("Task " + taskId)
    -            .addAllResources(cpuResourcesToUse.asJava)
    -            .addAllResources(memResourcesToUse.asJava)
    -
    -          sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image =>
    -            MesosSchedulerBackendUtil
    -              .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder)
    +        val availableResources = remainingResources(offerId)
    +        val offerMem = getResource(availableResources, "mem")
    +        val offerCpu = getResource(availableResources, "cpus")
    +
    +        // Catch offer limits
    +        calculateUsableResources(
    +          sc,
    +          offerCpu.toInt,
    +          offerMem.toInt
    +        ).flatMap(
    +          {
    +            // Catch "global" limits
    +            case (taskCPUs: Int, taskMemory: Int) =>
    +              if (numExecutors() >= executorLimit) {
    +                logTrace(s"${numExecutors()} exceeds limit of $executorLimit")
    +                None
    +              } else if (
    +                slaves.get(slaveId).map(_.taskFailures).getOrElse(0) >= MAX_SLAVE_FAILURES
    +              ) {
    +                logTrace(s"Slave $slaveId exceeded limit of $MAX_SLAVE_FAILURES failures")
    +                None
    +              } else {
    +                Some((taskCPUs, taskMemory))
    +              }
               }
    -
    -          tasks(offer.getId) ::= taskBuilder.build()
    -          remainingResources(offerId) = resourcesLeft.asJava
    -          totalCoresAcquired += taskCPUs
    -          coresByTaskId(taskId) = taskCPUs
    +        ) match {
    +          case Some((taskCPUs: Int, taskMemory: Int)) =>
    +            // Create a task
    +            launchTasks = true
    +            val taskId = newMesosTaskId()
    +
    +            slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
    +
    +            val (afterCPUResources, cpuResourcesToUse) =
    +              partitionResources(availableResources, "cpus", taskCPUs)
    +            val (resourcesLeft, memResourcesToUse) =
    +              partitionResources(afterCPUResources.asJava, "mem", taskMemory)
    +
    +            val taskBuilder = MesosTaskInfo.newBuilder()
    +              .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
    +              .setSlaveId(offer.getSlaveId)
    +              .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
    +              .setName("Task " + taskId)
    +              .addAllResources(cpuResourcesToUse.asJava)
    +              .addAllResources(memResourcesToUse.asJava)
    +
    +            sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image =>
    +              MesosSchedulerBackendUtil
    +                .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder)
    +            }
    +
    +            tasks(offer.getId) ::= taskBuilder.build()
    +            remainingResources(offerId) = resourcesLeft.asJava
    +            totalCoresAcquired += taskCPUs
    +            coresByTaskId(taskId) = taskCPUs
    +          case None => logDebug(s"Insufficient offer CPU: $offerCpu  MEM: $offerMem")
             }
           }
         }
         tasks.toMap
       }
     
    -  private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = {
    -    val offerMem = getResource(resources, "mem")
    -    val offerCPUs = getResource(resources, "cpus").toInt
    -    val cpus = executorCores(offerCPUs)
    -    val mem = executorMemory(sc)
    -
    -    cpus > 0 &&
    -      cpus <= offerCPUs &&
    -      cpus + totalCoresAcquired <= maxCores &&
    -      mem <= offerMem &&
    -      numExecutors() < executorLimit &&
    -      slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES
    +  /**
    +   * Try and fit the resources to the constraints. Return None if it does not match
    +   *
    +   * @param sc Spark context
    +   * @param availableCpus The available CPUs
    +   * @param availableMem The available Memory
    +   * @return Tuple of CPU (integer cores)  and Memory (integer MB) desired
    +   */
    +  private[mesos]
    +  def calculateUsableResources(sc: SparkContext, availableCpus: Int, availableMem: Int):
    +  Option[(Int, Int)] = {
    +    val desiredMemory = executorMemory(sc)
    +    val desiredCpu = executorCores(availableCpus)
    --- End diff --
    
    I can do that. Will be a few weeks before I can tackle this again though.


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

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

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


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    I broke something in the test suite, evaluating and I'll open this back in a bit


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

Posted by drcrallen <gi...@git.apache.org>.
GitHub user drcrallen reopened a pull request:

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

    [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedulerBackend offer consideration

    The offer acceptance workflow is a little hard to follow and not very extensible for future considerations for offers. This is a patch that makes the workflow a little more explicit in its handling of offer resources.

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

    $ git pull https://github.com/metamx/spark SPARK15992

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

    https://github.com/apache/spark/pull/13715.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 #13715
    
----

----


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    Thanks for the pull request, but the refactoring seems to have made it more difficult to read?



---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    **[Test build #60997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60997/consoleFull)** for PR 13715 at commit [`9e0aedf`](https://github.com/apache/spark/commit/9e0aedf12816c317db0a65e21adc921258608a4b).


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60997/
    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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

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

    https://github.com/apache/spark/pull/13715#discussion_r67988575
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala ---
    @@ -382,59 +382,97 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
           for (offer <- offers) {
             val slaveId = offer.getSlaveId.getValue
             val offerId = offer.getId.getValue
    -        val resources = remainingResources(offerId)
    -
    -        if (canLaunchTask(slaveId, resources)) {
    -          // Create a task
    -          launchTasks = true
    -          val taskId = newMesosTaskId()
    -          val offerCPUs = getResource(resources, "cpus").toInt
    -
    -          val taskCPUs = executorCores(offerCPUs)
    -          val taskMemory = executorMemory(sc)
    -
    -          slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
    -
    -          val (afterCPUResources, cpuResourcesToUse) =
    -            partitionResources(resources, "cpus", taskCPUs)
    -          val (resourcesLeft, memResourcesToUse) =
    -            partitionResources(afterCPUResources.asJava, "mem", taskMemory)
    -
    -          val taskBuilder = MesosTaskInfo.newBuilder()
    -            .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
    -            .setSlaveId(offer.getSlaveId)
    -            .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
    -            .setName("Task " + taskId)
    -            .addAllResources(cpuResourcesToUse.asJava)
    -            .addAllResources(memResourcesToUse.asJava)
    -
    -          sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image =>
    -            MesosSchedulerBackendUtil
    -              .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder)
    +        val availableResources = remainingResources(offerId)
    +        val offerMem = getResource(availableResources, "mem")
    +        val offerCpu = getResource(availableResources, "cpus")
    +
    +        // Catch offer limits
    +        calculateUsableResources(
    +          sc,
    +          offerCpu.toInt,
    +          offerMem.toInt
    +        ).flatMap(
    +          {
    +            // Catch "global" limits
    +            case (taskCPUs: Int, taskMemory: Int) =>
    +              if (numExecutors() >= executorLimit) {
    +                logTrace(s"${numExecutors()} exceeds limit of $executorLimit")
    +                None
    +              } else if (
    +                slaves.get(slaveId).map(_.taskFailures).getOrElse(0) >= MAX_SLAVE_FAILURES
    +              ) {
    +                logTrace(s"Slave $slaveId exceeded limit of $MAX_SLAVE_FAILURES failures")
    +                None
    +              } else {
    +                Some((taskCPUs, taskMemory))
    +              }
               }
    -
    -          tasks(offer.getId) ::= taskBuilder.build()
    -          remainingResources(offerId) = resourcesLeft.asJava
    -          totalCoresAcquired += taskCPUs
    -          coresByTaskId(taskId) = taskCPUs
    +        ) match {
    +          case Some((taskCPUs: Int, taskMemory: Int)) =>
    +            // Create a task
    +            launchTasks = true
    +            val taskId = newMesosTaskId()
    +
    +            slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId)
    +
    +            val (afterCPUResources, cpuResourcesToUse) =
    +              partitionResources(availableResources, "cpus", taskCPUs)
    +            val (resourcesLeft, memResourcesToUse) =
    +              partitionResources(afterCPUResources.asJava, "mem", taskMemory)
    +
    +            val taskBuilder = MesosTaskInfo.newBuilder()
    +              .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build())
    +              .setSlaveId(offer.getSlaveId)
    +              .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId))
    +              .setName("Task " + taskId)
    +              .addAllResources(cpuResourcesToUse.asJava)
    +              .addAllResources(memResourcesToUse.asJava)
    +
    +            sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image =>
    +              MesosSchedulerBackendUtil
    +                .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder)
    +            }
    +
    +            tasks(offer.getId) ::= taskBuilder.build()
    +            remainingResources(offerId) = resourcesLeft.asJava
    +            totalCoresAcquired += taskCPUs
    +            coresByTaskId(taskId) = taskCPUs
    +          case None => logDebug(s"Insufficient offer CPU: $offerCpu  MEM: $offerMem")
             }
           }
         }
         tasks.toMap
       }
     
    -  private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = {
    -    val offerMem = getResource(resources, "mem")
    -    val offerCPUs = getResource(resources, "cpus").toInt
    -    val cpus = executorCores(offerCPUs)
    -    val mem = executorMemory(sc)
    -
    -    cpus > 0 &&
    -      cpus <= offerCPUs &&
    -      cpus + totalCoresAcquired <= maxCores &&
    -      mem <= offerMem &&
    -      numExecutors() < executorLimit &&
    -      slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES
    +  /**
    +   * Try and fit the resources to the constraints. Return None if it does not match
    +   *
    +   * @param sc Spark context
    +   * @param availableCpus The available CPUs
    +   * @param availableMem The available Memory
    +   * @return Tuple of CPU (integer cores)  and Memory (integer MB) desired
    +   */
    +  private[mesos]
    +  def calculateUsableResources(sc: SparkContext, availableCpus: Int, availableMem: Int):
    +  Option[(Int, Int)] = {
    +    val desiredMemory = executorMemory(sc)
    +    val desiredCpu = executorCores(availableCpus)
    --- End diff --
    
    I like the idea, but it does make it harder to read. Can we refactor so we can collapse global and resource limits, and introduce like a case class for each condition, where it knows how to verify and logs error message when it doesn't match? 
    I think it makes adding and understanding how offers are matched better.



---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    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 issue #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrained...

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

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


---
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 #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    **[Test build #60997 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60997/consoleFull)** for PR 13715 at commit [`9e0aedf`](https://github.com/apache/spark/commit/9e0aedf12816c317db0a65e21adc921258608a4b).
     * 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 issue #13715: [SPARK-15992] [MESOS] Refactor MesosCoarseGrainedSchedul...

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

    https://github.com/apache/spark/pull/13715
  
    (ping @drcrallen)


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