You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yanji84 <gi...@git.apache.org> on 2017/03/01 02:04:53 UTC

[GitHub] spark pull request #17109: [SPARK-19740][MESOS]

GitHub user yanji84 opened a pull request:

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

    [SPARK-19740][MESOS]

    ## What changes were proposed in this pull request?
    
    Allow passing in arbitrary parameters into docker when launching spark executors on mesos with docker containerizer @tnachen
    
    ## How was this patch tested?
    
    Manually built and tested with passed in parameter

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

    $ git pull https://github.com/yanji84/spark ji/allow_set_docker_user

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

    https://github.com/apache/spark/pull/17109.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 #17109
    
----
commit 4f8368ea727e5689e96794884b8d1baf3eccb5d5
Author: Ji Yan <ji...@jis-macbook-air.local>
Date:   2017-02-25T22:36:31Z

    allow setting docker user when running spark on mesos with docker containerizer

commit bba57f9491703b4b06e82144a57660cbafa193ee
Author: Ji Yan <ji...@jis-macbook-air.local>
Date:   2017-02-26T01:34:49Z

    allow arbitrary parameters to pass to docker through spark conf

commit ae30e239e574cebc9774087e038aa0853d9939fc
Author: Ji Yan <ji...@jis-macbook-air.local>
Date:   2017-02-26T21:32:45Z

    add some debug prints

commit ecb7a8e87589d4b72fe836f91cdf4d8a7e5a53bc
Author: Ji Yan <ji...@jis-macbook-air.local>
Date:   2017-03-01T01:54:34Z

    remove debug print

----


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    LGTM @srowen ready to merge


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105457432
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -357,6 +357,16 @@ See the [configuration page](configuration.html) for information on Spark config
       </td>
     </tr>
     <tr>
    +  <td><code>spark.mesos.executor.docker.params</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Set the list of custom parameters which will be passed into the <code>docker run</code> command when launching the Spark executor on mesos using docker containerizer. The format of this property is a comma-separated list of
    --- End diff --
    
    s/mesos/Mesos
    
    s/docker containerizer/the docker containerizer


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73938/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #74392 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74392/testReport)** for PR 17109 at commit [`737acf0`](https://github.com/apache/spark/commit/737acf07ceea8f4bc92b9eaa8c572af19b2e0b88).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105724816
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler.cluster.mesos
    +
    +import org.scalatest._
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +
    +class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar {
    --- End diff --
    
    Is `Matchers` used anymore?


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

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


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r106052638
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    I am not very familiar with docker parameters - but wont this method not fail if '=' exists in the value ?
    Or is that prohibited by docker ?


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105457517
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -357,6 +357,16 @@ See the [configuration page](configuration.html) for information on Spark config
       </td>
     </tr>
     <tr>
    +  <td><code>spark.mesos.executor.docker.params</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Set the list of custom parameters which will be passed into the <code>docker run</code> command when launching the Spark executor on mesos using docker containerizer. The format of this property is a comma-separated list of
    +    key/value pairs. That is they take the form:
    --- End diff --
    
    s/That is they take the form:/Example:


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r111671944
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    sorry missed out this comment earlier, Ii set limit to 2, pushed the new pull request


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74488/
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105458256
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a comma-delimited list of arbitrary parameters, each of which
    +   * takes the form key=value
    +   */
    +  def parseParamsSpec(params: String): List[Parameter] = {
    --- End diff --
    
    private


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @yanji84 I have no experience with Mesos and am not, in general, a reviewer for this code, and don't follow Mesos changes. That's why. I pitch in to help merge but would appreciate others stepping up to help.


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r111045370
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    It should be quoted: https://github.com/docker/docker/issues/12763


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105458234
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a comma-delimited list of arbitrary parameters, each of which
    +   * takes the form key=value
    +   */
    +  def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { kv: Array[String] =>
    --- End diff --
    
    s/kv/parameter for naming consistency


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @srowen is there anything else holding this up? why does it take so long? 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73937 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73937/testReport)** for PR 17109 at commit [`4087936`](https://github.com/apache/spark/commit/408793696d354170c7944945caabda586ff22a7e).
     * 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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105458287
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a comma-delimited list of arbitrary parameters, each of which
    --- End diff --
    
    "list of docker parameters"


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105760039
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    to be consistent with other methods, i think we should stick with spec.
    
    >   def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = {
    >     portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] =>
    >       val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping
    
    it seems to be better to reserve the name parameter later for builder
    
    > val param: Parameter.Builder = Parameter.newBuilder()



---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @srowen @mgummelt PTAL


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105460213
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala ---
    @@ -0,0 +1,55 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler.cluster.mesos
    +
    +import org.scalatest._
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +
    +class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar {
    +
    +  test("Parse arbitrary parameter to pass into docker containerizer") {
    +    val parsed = MesosSchedulerBackendUtil.parseParamsSpec("a=1,b=2,c=3")
    +    parsed(0).getKey shouldBe "a"
    --- End diff --
    
    check out the other test suites.  We use `assert`, not `Matchers`


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73944 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73944/testReport)** for PR 17109 at commit [`0696d4f`](https://github.com/apache/spark/commit/0696d4f5bb43b5a6aec37700eb34dae375f611e9).
     * 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @srowen Appreciate the help you're doing, I think we're doing what we can to help review these patches and making sure Mesos support is still being maintained and improved over time. 
    If you trust our judgement and also us still around fixing issues when arises, then we really just need someone like you to help merge patches. 
    Ensuring someone else or if anyone that's been contributing to this area can become a committer definitely is a ever ongoing problem that we're still hoping one day can be addressed. Another parallel effort that I think is very worth investigating is to decouple the cluster manager intergation from Spark, which I believe is becoming more relevant now as we have more integration coming.
    
    Long story short, if you can still help in the mean time will be greatly appreciated as we can still make sure improvements around Mesos integration can still happen


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Is this one still good to go? I see a late comment above. I can merge it, no problem.


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #74392 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74392/testReport)** for PR 17109 at commit [`737acf0`](https://github.com/apache/spark/commit/737acf07ceea8f4bc92b9eaa8c572af19b2e0b88).
     * 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #75828 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75828/testReport)** for PR 17109 at commit [`2f3f8b2`](https://github.com/apache/spark/commit/2f3f8b20942e8d24d582ad69a6ece1377b33ec65).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @tnachen I will add 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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #74299 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74299/testReport)** for PR 17109 at commit [`03e89eb`](https://github.com/apache/spark/commit/03e89eb2bae3b08207429a6b772d7dcae45b554c).
     * 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #74488 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74488/testReport)** for PR 17109 at commit [`cbb784a`](https://github.com/apache/spark/commit/cbb784a1a278f2d0db5c5122d52c30dfd26fc3db).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class MesosSchedulerBackendUtilSuite extends SparkFunSuite `


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73938 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73938/testReport)** for PR 17109 at commit [`54f9ec8`](https://github.com/apache/spark/commit/54f9ec802002e702f1c3954d97be47bcdb5f7786).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar `


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105763729
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    ok


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73645 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73645/testReport)** for PR 17109 at commit [`ecb7a8e`](https://github.com/apache/spark/commit/ecb7a8e87589d4b72fe836f91cdf4d8a7e5a53bc).
     * 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73937/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73942/
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105460407
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala ---
    @@ -0,0 +1,55 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler.cluster.mesos
    +
    +import org.scalatest._
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +
    +class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar {
    +
    +  test("Parse arbitrary parameter to pass into docker containerizer") {
    +    val parsed = MesosSchedulerBackendUtil.parseParamsSpec("a=1,b=2,c=3")
    +    parsed(0).getKey shouldBe "a"
    +    parsed(0).getValue shouldBe "1"
    +    parsed(1).getKey shouldBe "b"
    +    parsed(1).getValue shouldBe "2"
    +    parsed(2).getKey shouldBe "c"
    +    parsed(2).getValue shouldBe "3"
    +
    +    val invalid = MesosSchedulerBackendUtil.parseParamsSpec("a,b")
    --- End diff --
    
    invalid entries should be in a separate 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 pull request #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105724131
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    s/spec/parameter, for consistency


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73944 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73944/testReport)** for PR 17109 at commit [`0696d4f`](https://github.com/apache/spark/commit/0696d4f5bb43b5a6aec37700eb34dae375f611e9).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105460513
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala ---
    @@ -0,0 +1,55 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler.cluster.mesos
    +
    +import org.scalatest._
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +
    +class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar {
    +
    +  test("Parse arbitrary parameter to pass into docker containerizer") {
    +    val parsed = MesosSchedulerBackendUtil.parseParamsSpec("a=1,b=2,c=3")
    --- End diff --
    
    This should be a private method, so let's not test it.  Just test `containerInfo`


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @srowen @tnachen confirm merge


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #74299 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74299/testReport)** for PR 17109 at commit [`03e89eb`](https://github.com/apache/spark/commit/03e89eb2bae3b08207429a6b772d7dcae45b554c).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Merged to master


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @tnachen does the PR look good? 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73937 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73937/testReport)** for PR 17109 at commit [`4087936`](https://github.com/apache/spark/commit/408793696d354170c7944945caabda586ff22a7e).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @srowen We do appreciate your help with Mesos commits, and generally find you responsive.  I have a habit of pinging you for merges because you seemed to have stepped in once @andrewor14 stepped away.  There are really only two active Spark contributers with knowledge of Mesos code, being myself and @skonto.  We would obviously love it if one of us could be made a committer so we could more effectively maintain the Mesos module.  It's something I've spoked with @rxin about previously, but there was some concern about my total code volume.  I'll be upstreaming a sizable feature this week (Kerberos support on Mesos).  Maybe that, along with my 1.5 year history of responsive code reviews and contributions, would push me over the edge? 


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @mgummelt comments addressed, please take another look


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

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


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73944/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #75828 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75828/testReport)** for PR 17109 at commit [`2f3f8b2`](https://github.com/apache/spark/commit/2f3f8b20942e8d24d582ad69a6ece1377b33ec65).
     * 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75828/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74392/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    hello, any update on 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @yanji84 can you add a test for 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105457620
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -357,6 +357,16 @@ See the [configuration page](configuration.html) for information on Spark config
       </td>
     </tr>
     <tr>
    +  <td><code>spark.mesos.executor.docker.params</code></td>
    --- End diff --
    
    s/params/parameters
    
    to be consistent with the Mesos protobuf terminology


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74299/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73938 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73938/testReport)** for PR 17109 at commit [`54f9ec8`](https://github.com/apache/spark/commit/54f9ec802002e702f1c3954d97be47bcdb5f7786).


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73942 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73942/testReport)** for PR 17109 at commit [`423dfa8`](https://github.com/apache/spark/commit/423dfa8dd612df922bc977da6cde04bb9473fe65).
     * This patch **fails Scala style 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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r105759500
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler.cluster.mesos
    +
    +import org.scalatest._
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +
    +class MesosSchedulerBackendUtilSuite extends SparkFunSuite with Matchers with MockitoSugar {
    --- End diff --
    
    i'll check, if not, will remove it


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r111670608
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    I see, so we should split with a limit instead.
    @yanji84 can you fix 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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73645/
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

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


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    Hey sorry for the late response, code looks good to me, however we need to add documentation about the new flag. Can you modify the Mesos configuration docs in the docs folder?


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    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 issue #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    @tnachen test ready


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass ...

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

    https://github.com/apache/spark/pull/17109#discussion_r106053115
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -99,6 +99,26 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    +  /**
    +   * Parse a list of docker parameters, each of which
    +   * takes the form key=value
    +   */
    +  private def parseParamsSpec(params: String): List[Parameter] = {
    +    params.split(",").map(_.split("=")).flatMap { spec: Array[String] =>
    --- End diff --
    
    hmm if a value contains a '=' we will have a parsing error.


---
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 #17109: [SPARK-19740][MESOS]Add support in Spark to pass arbitra...

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

    https://github.com/apache/spark/pull/17109
  
    **[Test build #73942 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73942/testReport)** for PR 17109 at commit [`423dfa8`](https://github.com/apache/spark/commit/423dfa8dd612df922bc977da6cde04bb9473fe65).


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