You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by susanxhuynh <gi...@git.apache.org> on 2017/10/05 12:20:53 UTC

[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

GitHub user susanxhuynh opened a pull request:

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

    [SPARK-22131][MESOS] Mesos driver secrets

    ## Background
    
    In #18837 , @ArtRand added Mesos secrets support to the dispatcher. **This PR is to add the same secrets support to the drivers.** This means if the secret configs are set, the driver will launch executors that have access to either env or file-based secrets.
    
    One use case for this is to support TLS in the driver <=> executor communication.
    
    ## What changes were proposed in this pull request?
    
    Most of the changes are a refactor of the dispatcher secrets support (#18837) - moving it to a common place that can be used by both the dispatcher and drivers. The same goes for the unit tests.
    
    ## How was this patch tested?
    
    There are four config combinations: [env or file-based] x [value or reference secret]. For each combination:
    - Added a unit test.
    - Tested in DC/OS.


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

    $ git pull https://github.com/mesosphere/spark sh-mesos-driver-secret

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

    https://github.com/apache/spark/pull/19437.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 #19437
    
----
commit b289bcc95f0b67cda94ddf416fc9a15e5d1855b4
Author: Susan X. Huynh <xh...@mesosphere.com>
Date:   2017-10-04T11:30:31Z

    [SPARK-22131] Mesos driver secrets. The driver launches executors that have access to env or file-based secrets.

commit 6f062c00f6382d266619b4a56a753ec27d1db10b
Author: Susan X. Huynh <xh...@mesosphere.com>
Date:   2017-10-05T12:07:20Z

    [SPARK-22131] Updated docs

----


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82885 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82885/testReport)** for PR 19437 at commit [`a801799`](https://github.com/apache/spark/commit/a80179953b38ec5e7f015eadc1b48075f01e4b65).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144681758
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    --- End diff --
    
    Indentation? 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143312857
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,7 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def containerInfo(conf: SparkConf, secretConfig: MesosSecretConfig): ContainerInfo.Builder = {
    --- End diff --
    
    IMHO a better name would be createContainerInfo or buildContainerInfo


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merging to master.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680353
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf, secretConfig: MesosSecretConfig):
    --- End diff --
    
    Maybe change `secretConfig` to `mesosConfig` and pass the whole thing? That way if we want to add new functionality later this function is more general. Given that most of what we do is proto-generation, I bet we'll have to do this eventually anyways. 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143312903
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -17,10 +17,14 @@
     
     package org.apache.spark.scheduler.cluster.mesos
     
    -import org.apache.mesos.Protos.{ContainerInfo, Image, NetworkInfo, Parameter, Volume}
    +import org.apache.mesos.Protos._
    --- End diff --
    
    avoid wild card imports if possible.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146681961
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala ---
    @@ -21,6 +21,39 @@ import java.util.concurrent.TimeUnit
     
     import org.apache.spark.internal.config.ConfigBuilder
     
    +private[spark] class MesosSecretConfig(taskType: String) {
    --- End diff --
    
    Just to avoid accidentally trying to instantiate this class in other places, make the constructor private.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146702191
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    +    by spark.mesos.executor.secret.names or spark.mesos.executor.secret.values will be
    +    set to the provided environment variable in the executor's process. Example:
    +    
    +    <pre>ENVKEY1,ENVKEY2</pre>
    +  </td>
    +  </tr>
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146702357
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    +    by spark.mesos.executor.secret.names or spark.mesos.executor.secret.values will be
    +    set to the provided environment variable in the executor's process. Example:
    +    
    +    <pre>ENVKEY1,ENVKEY2</pre>
    +  </td>
    +  </tr>
    +  <tr>
    +<td><code>spark.mesos.executor.secret.filenames</code></td>
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146680234
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    --- End diff --
    
    I cannot really understand this description.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82565 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82565/testReport)** for PR 19437 at commit [`73b2cbf`](https://github.com/apache/spark/commit/73b2cbfeb43a3448a570c055d62c439e267490be).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143565549
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +174,122 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAME).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUE).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], s: Seq[Secret]): Boolean = {
    +    if (dest.isEmpty) {  // no destination set (ie not using secrets of this type
    --- End diff --
    
    Good point. If they specify paths but no secrets, it should throw an exception.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146681289
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    --- End diff --
    
    This "protobuf" thing is mentioned in a bunch of places but there's no link nor explanation of what it is. Are Mesos users just supposed to know what that is?


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145294156
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    +        val source = Volume.Source.newBuilder()
    +          .setType(Volume.Source.Type.SECRET)
    +          .setSecret(s)
    +        Volume.newBuilder()
    +          .setContainerPath(p)
    +          .setSource(source)
    +          .setMode(Volume.Mode.RO)
    +          .build
    +    }.toList
    +  }
    +
    +  def getSecretEnvVar(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  List[Variable] = {
    --- End diff --
    
    indentation? 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680478
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    --- End diff --
    
    Need a space at the end of this log line (my bad!)


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146989708
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    +    <code>spark.mesos.driver.secret.names</code>,
    +    <code>spark.mesos.executor.secret.values</code>,
    +    <code>spark.mesos.executor.secret.names</code>,
       </td>
    -  </tr>
    -  <tr>
    -<td><code>spark.mesos.driver.secret.filenames</code></td>
       <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced by
    -    spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    written to the provided file. Paths are relative to the container's work
    -    directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    A secret is specified by its contents and destination. These properties
    +    specify a secret's contents. To specify a secret's destination, see the cell below.
    +
    +    You can specify a secret's contents either (1) by value or (2) by reference.
    +    (1) To specify a secret by value, set the
    +    <code>spark.mesos.[driver|executor].secret.values</code>
    +    property, to make the secret available in the driver or executors.
    +    For example, to make a secret password "guessme" available to the driver process, set:
    +    
    +    <pre>spark.mesos.driver.secret.values=guessme</pre>
    +    
    +    (2) To specify a secret that has been placed in a secret store
    --- End diff --
    
    What is this secret store? I don't remember Spark providing this feature, so it's not clear how this secret store is configured or used by Spark.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146680539
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    +    by spark.mesos.executor.secret.names or spark.mesos.executor.secret.values will be
    +    set to the provided environment variable in the executor's process. Example:
    +    
    +    <pre>ENVKEY1,ENVKEY2</pre>
    +  </td>
    +  </tr>
    --- End diff --
    
    Indentation doesn't match other rows.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82867 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82867/testReport)** for PR 19437 at commit [`770d307`](https://github.com/apache/spark/commit/770d307bd67796b93f20d7dba105905059af7a0e).


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146991701
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +177,88 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    --- End diff --
    
    "Cannot specify both value-type and reference-type secrets."


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144682312
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -17,10 +17,14 @@
     
     package org.apache.spark.scheduler.cluster.mesos
    --- End diff --
    
    Out of curiosity, why do we have this file _and_ `MesosSchedulerUtils`?


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145437791
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf):
    +  ContainerInfo.Builder = {
    --- End diff --
    
    Fixed


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146682161
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala ---
    @@ -394,9 +393,10 @@ private[spark] class MesosClusterScheduler(
         }
     
         // add secret environment variables
    -    getSecretEnvVar(desc).foreach { variable =>
    +    MesosSchedulerBackendUtil.getSecretEnvVar(desc.conf, config.driverSecretConfig)
    +      .foreach { variable =>
           if (variable.getSecret.getReference.isInitialized) {
    --- End diff --
    
    This block now needs to be indented further.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @vanzin Would you mind reviewing this PR? A followup to ArtRand's secrets PR.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @susanxhuynh @skonto The secret-containing protos will be valid in Mesos 1.3 onwards, thus why the scheduler has that requirement. DC/OS with file-based secrets has Mesos 1.4 thus why we test it there. 


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r147162791
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    +    <code>spark.mesos.driver.secret.names</code>,
    +    <code>spark.mesos.executor.secret.values</code>,
    +    <code>spark.mesos.executor.secret.names</code>,
       </td>
    -  </tr>
    -  <tr>
    -<td><code>spark.mesos.driver.secret.filenames</code></td>
       <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced by
    -    spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    written to the provided file. Paths are relative to the container's work
    -    directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    A secret is specified by its contents and destination. These properties
    +    specify a secret's contents. To specify a secret's destination, see the cell below.
    +
    +    You can specify a secret's contents either (1) by value or (2) by reference.
    +    (1) To specify a secret by value, set the
    +    <code>spark.mesos.[driver|executor].secret.values</code>
    +    property, to make the secret available in the driver or executors.
    +    For example, to make a secret password "guessme" available to the driver process, set:
    +    
    +    <pre>spark.mesos.driver.secret.values=guessme</pre>
    +    
    +    (2) To specify a secret that has been placed in a secret store
    --- End diff --
    
    I added a note about this. The secret store has to be provided by the user. You can configure Mesos to integrate with a secret store via a custom Mesos module.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #83085 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83085/testReport)** for PR 19437 at commit [`88dfb42`](https://github.com/apache/spark/commit/88dfb42bfd794ab1eea969c3b51970e68e4ca407).


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @ArtRand @skonto Please review. Tests passed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #83049 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83049/testReport)** for PR 19437 at commit [`3cd1ae8`](https://github.com/apache/spark/commit/3cd1ae82e136b42059091213f21e4649d267cba4).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145286357
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145290603
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    --- End diff --
    
    Fixed


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143413006
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala ---
    @@ -21,6 +21,39 @@ import java.util.concurrent.TimeUnit
     
     import org.apache.spark.internal.config.ConfigBuilder
     
    +private[spark] class MesosSecretConfig(taskType: String) {
    +  private[spark] val SECRET_NAME =
    --- End diff --
    
    shouldn't be SECRET_NAMES? 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143543546
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala ---
    @@ -21,6 +21,39 @@ import java.util.concurrent.TimeUnit
     
     import org.apache.spark.internal.config.ConfigBuilder
     
    +private[spark] class MesosSecretConfig(taskType: String) {
    +  private[spark] val SECRET_NAME =
    --- End diff --
    
    I see what you mean. I'll change them.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146991758
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +177,88 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    --- End diff --
    
    nit (also in other places): `.map { s => ... }`


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @srowen Ping, would you like to help review?


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r147161356
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    +    <code>spark.mesos.driver.secret.names</code>,
    +    <code>spark.mesos.executor.secret.values</code>,
    +    <code>spark.mesos.executor.secret.names</code>,
       </td>
    -  </tr>
    -  <tr>
    -<td><code>spark.mesos.driver.secret.filenames</code></td>
       <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced by
    -    spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    written to the provided file. Paths are relative to the container's work
    -    directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    A secret is specified by its contents and destination. These properties
    +    specify a secret's contents. To specify a secret's destination, see the cell below.
    +
    +    You can specify a secret's contents either (1) by value or (2) by reference.
    --- End diff --
    
    Good catch. I added `<p>` paragraph tags to separate paragraphs in this table cell. (Jekyll doesn't need them in normal text body, only inside a table cell.)


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r147159788
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    --- End diff --
    
    It looks okay in a browser. This column takes up about 25% of the page width. Here's a screenshot: 
    ![documentation screenshot](https://user-images.githubusercontent.com/22622418/32058932-76e5e456-ba1f-11e7-999e-4302d49e636c.png)



---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146702275
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    --- End diff --
    
    Let me try to improve the descriptions of all these secret configs.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r147165029
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +177,88 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143221450
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala ---
    @@ -105,4 +108,108 @@ object Utils {
       def createTaskId(taskId: String): TaskID = {
         TaskID.newBuilder().setValue(taskId).build()
       }
    +
    +  def configEnvBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretName = "/path/to/secret,/anothersecret"
    +    val envKey = "SECRET_ENV_KEY,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_NAME.key -> secretName,
    +      secretConfig.SECRET_ENVKEY.key -> envKey
    +    )
    +  }
    +
    +  def verifyEnvBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    --- End diff --
    
    assert(envVars.count(!_.getName.startsWith("SPARK_")) == 2)


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @skonto I haven't tested with TLS; I'll work on that in the next couple of days.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146683210
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    +        val source = Volume.Source.newBuilder()
    +          .setType(Volume.Source.Type.SECRET)
    +          .setSecret(s)
    +        Volume.newBuilder()
    +          .setContainerPath(p)
    +          .setSource(source)
    +          .setMode(Volume.Mode.RO)
    +          .build
    +    }.toList
    +  }
    +
    +  def getSecretEnvVar(conf: SparkConf, secretConfig: MesosSecretConfig):
    +    List[Variable] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretEnvKeys = conf.get(secretConfig.SECRET_ENVKEYS).getOrElse(Nil)
    +    if (illegalSecretInput(secretEnvKeys, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and environment keys " +
    +          s"for environment-based reference secrets got secrets $secrets, " +
    +          s"and keys $secretEnvKeys")
    +    }
    +
    +    secrets.zip(secretEnvKeys).map {
    +      case (s, k) =>
    --- End diff --
    
    Move to previous line.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145437403
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,74 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    --- End diff --
    
    Mentioned custom module and added link to Mesos documentation.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680608
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -522,6 +522,43 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    --- End diff --
    
    What do you think about putting an example here like we do for `spark.mesos.network.labels` - something general for all secrets? 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143540778
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala ---
    @@ -105,4 +108,108 @@ object Utils {
       def createTaskId(taskId: String): TaskID = {
         TaskID.newBuilder().setValue(taskId).build()
       }
    +
    +  def configEnvBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretName = "/path/to/secret,/anothersecret"
    +    val envKey = "SECRET_ENV_KEY,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_NAME.key -> secretName,
    +      secretConfig.SECRET_ENVKEY.key -> envKey
    +    )
    +  }
    +
    +  def verifyEnvBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    --- End diff --
    
    Thanks, I'll fix it.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145079551
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -17,10 +17,14 @@
     
     package org.apache.spark.scheduler.cluster.mesos
    --- End diff --
    
    Its for the MesosScheduler's helper functions but there are methods (eg. createResource) used all over the place.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145294074
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -17,10 +17,14 @@
     
     package org.apache.spark.scheduler.cluster.mesos
    --- End diff --
    
    Ah ok. One of these days let's make a "clean up" JIRA and harmonize all of this code. The naming is also all over the place.. 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146712778
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    --- End diff --
    
    ok


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146712170
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala ---
    @@ -394,9 +393,10 @@ private[spark] class MesosClusterScheduler(
         }
     
         // add secret environment variables
    -    getSecretEnvVar(desc).foreach { variable =>
    +    MesosSchedulerBackendUtil.getSecretEnvVar(desc.conf, config.driverSecretConfig)
    +      .foreach { variable =>
           if (variable.getSecret.getReference.isInitialized) {
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145290625
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    --- End diff --
    
    Fixed


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146682890
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf):
    +    ContainerInfo.Builder = {
    --- End diff --
    
    Move to previous line.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143571471
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala ---
    @@ -105,4 +108,108 @@ object Utils {
       def createTaskId(taskId: String): TaskID = {
         TaskID.newBuilder().setValue(taskId).build()
       }
    +
    +  def configEnvBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretName = "/path/to/secret,/anothersecret"
    +    val envKey = "SECRET_ENV_KEY,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_NAME.key -> secretName,
    +      secretConfig.SECRET_ENVKEY.key -> envKey
    +    )
    +  }
    +
    +  def verifyEnvBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    +    val variableOne = envVars.filter(_.getName == "SECRET_ENV_KEY").head
    +    assert(variableOne.getSecret.isInitialized)
    +    assert(variableOne.getSecret.getType == Secret.Type.REFERENCE)
    +    assert(variableOne.getSecret.getReference.getName == "/path/to/secret")
    +    assert(variableOne.getType == Environment.Variable.Type.SECRET)
    +    val variableTwo = envVars.filter(_.getName == "PASSWORD").head
    +    assert(variableTwo.getSecret.isInitialized)
    +    assert(variableTwo.getSecret.getType == Secret.Type.REFERENCE)
    +    assert(variableTwo.getSecret.getReference.getName == "/anothersecret")
    +    assert(variableTwo.getType == Environment.Variable.Type.SECRET)
    +  }
    +
    +  def configEnvBasedValueSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretValues = "user,password"
    +    val envKeys = "USER,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_VALUE.key -> secretValues,
    +      secretConfig.SECRET_ENVKEY.key -> envKeys
    +    )
    +  }
    +
    +  def verifyEnvBasedValueSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    --- End diff --
    
    Done


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145287767
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    --- End diff --
    
    I've removed this method. To be more consistent, I've moved this code back into MesosClusterScheduler. There's a little duplication, because MesosCoarseGrainedSchedulerBackend now has a similar code snippet, but it does avoid the mutation.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146683130
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    --- End diff --
    
    Move to previous line.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @srowen Would you like to help review? Adding Mesos secrets support in driver for executor tasks.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #83049 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83049/testReport)** for PR 19437 at commit [`3cd1ae8`](https://github.com/apache/spark/commit/3cd1ae82e136b42059091213f21e4649d267cba4).


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146682439
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala ---
    @@ -440,6 +420,23 @@ private[spark] class MesosClusterScheduler(
           CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build())
       }
     
    +  private def getContainerInfo(desc: MesosDriverDescription): ContainerInfo.Builder = {
    +    val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo(desc.conf)
    +
    +    MesosSchedulerBackendUtil.getSecretVolume(desc.conf, config.driverSecretConfig)
    +      .foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    --- End diff --
    
    Indent further.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145282997
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf, secretConfig: MesosSecretConfig):
    --- End diff --
    
    I've removed that second parameter and moved the secret stuff out of this method, since it's called by Fine-Grained mode, and we don't want to touch Fine-Grained mode.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143549493
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +174,122 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAME).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUE).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], s: Seq[Secret]): Boolean = {
    --- End diff --
    
    Done


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145079127
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala ---
    @@ -159,7 +160,8 @@ private[spark] class MesosFineGrainedSchedulerBackend(
           .setCommand(command)
           .setData(ByteString.copyFrom(createExecArg()))
     
    -    executorInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
    +    executorInfo.setContainer(
    --- End diff --
    
    +1 There is a PR for removing it already waiting for spark 3.0. Let's to give users the wrong impression that we maintain it.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680448
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  ContainerInfo.Builder = {
         val containerType = if (conf.contains("spark.mesos.executor.docker.image") &&
    --- End diff --
    
    Should probably have a check here for if secrets are present, because I don't think that secrets will work if you're _not_ also using the Mesos containerizer. 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145282300
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala ---
    @@ -159,7 +160,8 @@ private[spark] class MesosFineGrainedSchedulerBackend(
           .setCommand(command)
           .setData(ByteString.copyFrom(createExecArg()))
     
    -    executorInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
    +    executorInfo.setContainer(
    --- End diff --
    
    Okay, I've reverted this change in fine-grained mode.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143415419
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +174,122 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAME).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUE).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], s: Seq[Secret]): Boolean = {
    +    if (dest.isEmpty) {  // no destination set (ie not using secrets of this type
    --- End diff --
    
    I think this is redundant, it is covered by the last false statement anyway.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145294352
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,74 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    --- End diff --
    
    I'm not sure what the policy should be on this, but IIUC file-based secrets does require a backend module. Should we mention that here? 


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @vanzin Ping, would you mind reviewing this PR?


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146988600
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    +    <code>spark.mesos.driver.secret.names</code>,
    +    <code>spark.mesos.executor.secret.values</code>,
    +    <code>spark.mesos.executor.secret.names</code>,
       </td>
    -  </tr>
    -  <tr>
    -<td><code>spark.mesos.driver.secret.filenames</code></td>
       <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced by
    -    spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    written to the provided file. Paths are relative to the container's work
    -    directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    A secret is specified by its contents and destination. These properties
    +    specify a secret's contents. To specify a secret's destination, see the cell below.
    +
    +    You can specify a secret's contents either (1) by value or (2) by reference.
    --- End diff --
    
    Just want to confirm this cell renders as expected, and not in a single paragraph or something odd like that. I'm not really familiar with how jekyll translates this into HTML.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680489
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    --- End diff --
    
    Space at the end of this log line too.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82475 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82475/testReport)** for PR 19437 at commit [`6f062c0`](https://github.com/apache/spark/commit/6f062c00f6382d266619b4a56a753ec27d1db10b).


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    @susanxhuynh this a mesos 1.4 feature, shouldn't we document this for users?
    https://issues.apache.org/jira/browse/MESOS-7418


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #83085 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/83085/testReport)** for PR 19437 at commit [`88dfb42`](https://github.com/apache/spark/commit/88dfb42bfd794ab1eea969c3b51970e68e4ca407).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82868 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82868/testReport)** for PR 19437 at commit [`b2a3675`](https://github.com/apache/spark/commit/b2a36753b41820ec5e2d85a6a29fd8677bc0029a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145294610
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf):
    +  ContainerInfo.Builder = {
    --- End diff --
    
    indentation?


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r147165077
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +177,88 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146701899
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    --- End diff --
    
    I have grouped some of these properties together to make the description clearer. Please let me know what you think.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146712893
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    +        val source = Volume.Source.newBuilder()
    +          .setType(Volume.Source.Type.SECRET)
    +          .setSecret(s)
    +        Volume.newBuilder()
    +          .setContainerPath(p)
    +          .setSource(source)
    +          .setMode(Volume.Mode.RO)
    +          .build
    +    }.toList
    +  }
    +
    +  def getSecretEnvVar(conf: SparkConf, secretConfig: MesosSecretConfig):
    +    List[Variable] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretEnvKeys = conf.get(secretConfig.SECRET_ENVKEYS).getOrElse(Nil)
    +    if (illegalSecretInput(secretEnvKeys, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and environment keys " +
    +          s"for environment-based reference secrets got secrets $secrets, " +
    +          s"and keys $secretEnvKeys")
    +    }
    +
    +    secrets.zip(secretEnvKeys).map {
    +      case (s, k) =>
    --- End diff --
    
    ok


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145268821
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -522,6 +522,43 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    --- End diff --
    
    I'll add some examples. I didn't find a good general example; I added a separate example to each config option.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144680213
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala ---
    @@ -159,7 +160,8 @@ private[spark] class MesosFineGrainedSchedulerBackend(
           .setCommand(command)
           .setData(ByteString.copyFrom(createExecArg()))
     
    -    executorInfo.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf))
    +    executorInfo.setContainer(
    --- End diff --
    
    I'd almost prefer that we don't add any features to fine-grained right now. As we have virtually no test coverage on whether or not this will work. 


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143221337
  
    --- Diff: resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala ---
    @@ -105,4 +108,108 @@ object Utils {
       def createTaskId(taskId: String): TaskID = {
         TaskID.newBuilder().setValue(taskId).build()
       }
    +
    +  def configEnvBasedRefSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretName = "/path/to/secret,/anothersecret"
    +    val envKey = "SECRET_ENV_KEY,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_NAME.key -> secretName,
    +      secretConfig.SECRET_ENVKEY.key -> envKey
    +    )
    +  }
    +
    +  def verifyEnvBasedRefSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    +    val variableOne = envVars.filter(_.getName == "SECRET_ENV_KEY").head
    +    assert(variableOne.getSecret.isInitialized)
    +    assert(variableOne.getSecret.getType == Secret.Type.REFERENCE)
    +    assert(variableOne.getSecret.getReference.getName == "/path/to/secret")
    +    assert(variableOne.getType == Environment.Variable.Type.SECRET)
    +    val variableTwo = envVars.filter(_.getName == "PASSWORD").head
    +    assert(variableTwo.getSecret.isInitialized)
    +    assert(variableTwo.getSecret.getType == Secret.Type.REFERENCE)
    +    assert(variableTwo.getSecret.getReference.getName == "/anothersecret")
    +    assert(variableTwo.getType == Environment.Variable.Type.SECRET)
    +  }
    +
    +  def configEnvBasedValueSecrets(secretConfig: MesosSecretConfig): Map[String, String] = {
    +    val secretValues = "user,password"
    +    val envKeys = "USER,PASSWORD"
    +    Map(
    +      secretConfig.SECRET_VALUE.key -> secretValues,
    +      secretConfig.SECRET_ENVKEY.key -> envKeys
    +    )
    +  }
    +
    +  def verifyEnvBasedValueSecrets(launchedTasks: List[TaskInfo]): Unit = {
    +    val envVars = launchedTasks.head
    +      .getCommand
    +      .getEnvironment
    +      .getVariablesList
    +      .asScala
    +    assert(envVars
    +      .filter(!_.getName.startsWith("SPARK_")).length == 2)  // user-defined secret env vars
    --- End diff --
    
    assert(envVars.count(!_.getName.startsWith("SPARK_")) == 2)


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146712337
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala ---
    @@ -440,6 +420,23 @@ private[spark] class MesosClusterScheduler(
           CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build())
       }
     
    +  private def getContainerInfo(desc: MesosDriverDescription): ContainerInfo.Builder = {
    +    val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo(desc.conf)
    +
    +    MesosSchedulerBackendUtil.getSecretVolume(desc.conf, config.driverSecretConfig)
    +      .foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    --- End diff --
    
    Fixed.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143411366
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +174,122 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    +      envBuilder: Environment.Builder,
    +      conf: SparkConf,
    +      secretConfig: MesosSecretConfig): Unit = {
    +    getSecretEnvVar(conf, secretConfig).foreach { variable =>
    +      if (variable.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${variable.getSecret.getReference.getName}" +
    +          s"on file ${variable.getName}")
    +      } else {
    +        logInfo(s"Setting secret on environment variable name=${variable.getName}")
    +      }
    +      envBuilder.addVariables(variable)
    +    }
    +  }
    +
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAME).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUE).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], s: Seq[Secret]): Boolean = {
    --- End diff --
    
    s -> secrets


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146704074
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala ---
    @@ -21,6 +21,39 @@ import java.util.concurrent.TimeUnit
     
     import org.apache.spark.internal.config.ConfigBuilder
     
    +private[spark] class MesosSecretConfig(taskType: String) {
    --- End diff --
    
    Done. I've moved this `MesosSecretConfig` class inside the `config` object, and made the constructor private to that object.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r145437968
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -173,6 +178,90 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         containerInfo
       }
     
    +  private def getSecrets(conf: SparkConf, secretConfig: MesosSecretConfig): Seq[Secret] = {
    +    def createValueSecret(data: String): Secret = {
    +      Secret.newBuilder()
    +        .setType(Secret.Type.VALUE)
    +        .setValue(Secret.Value.newBuilder().setData(ByteString.copyFrom(data.getBytes)))
    +        .build()
    +    }
    +
    +    def createReferenceSecret(name: String): Secret = {
    +      Secret.newBuilder()
    +        .setReference(Secret.Reference.newBuilder().setName(name))
    +        .setType(Secret.Type.REFERENCE)
    +        .build()
    +    }
    +
    +    val referenceSecrets: Seq[Secret] =
    +      conf.get(secretConfig.SECRET_NAMES).getOrElse(Nil).map(s => createReferenceSecret(s))
    +
    +    val valueSecrets: Seq[Secret] = {
    +      conf.get(secretConfig.SECRET_VALUES).getOrElse(Nil).map(s => createValueSecret(s))
    +    }
    +
    +    if (valueSecrets.nonEmpty && referenceSecrets.nonEmpty) {
    +      throw new SparkException("Cannot specify VALUE type secrets and REFERENCE types ones")
    +    }
    +
    +    if (referenceSecrets.nonEmpty) referenceSecrets else valueSecrets
    +  }
    +
    +  private def illegalSecretInput(dest: Seq[String], secrets: Seq[Secret]): Boolean = {
    +    if (dest.nonEmpty) {
    +      // make sure there is a one-to-one correspondence between destinations and secrets
    +      if (dest.length != secrets.length) {
    +        return true
    +      }
    +    }
    +    false
    +  }
    +
    +  def getSecretVolume(conf: SparkConf, secretConfig: MesosSecretConfig): List[Volume] = {
    +    val secrets = getSecrets(conf, secretConfig)
    +    val secretPaths: Seq[String] =
    +      conf.get(secretConfig.SECRET_FILENAMES).getOrElse(Nil)
    +
    +    if (illegalSecretInput(secretPaths, secrets)) {
    +      throw new SparkException(
    +        s"Need to give equal numbers of secrets and file paths for file-based " +
    +          s"reference secrets got secrets $secrets, and paths $secretPaths")
    +    }
    +
    +    secrets.zip(secretPaths).map {
    +      case (s, p) =>
    +        val source = Volume.Source.newBuilder()
    +          .setType(Volume.Source.Type.SECRET)
    +          .setSecret(s)
    +        Volume.newBuilder()
    +          .setContainerPath(p)
    +          .setSource(source)
    +          .setMode(Volume.Mode.RO)
    +          .build
    +    }.toList
    +  }
    +
    +  def getSecretEnvVar(conf: SparkConf, secretConfig: MesosSecretConfig):
    +  List[Variable] = {
    --- End diff --
    
    Fixed


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82475 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82475/testReport)** for PR 19437 at commit [`6f062c0`](https://github.com/apache/spark/commit/6f062c00f6382d266619b4a56a753ec27d1db10b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146680903
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    +    by spark.mesos.executor.secret.names or spark.mesos.executor.secret.values will be
    +    set to the provided environment variable in the executor's process. Example:
    +    
    +    <pre>ENVKEY1,ENVKEY2</pre>
    +  </td>
    +  </tr>
    +  <tr>
    +<td><code>spark.mesos.executor.secret.filenames</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced by
    --- End diff --
    
    This is another description that is very weirdly worded. What I can understand of this:
    
    "A comma-separated list of file names to which secret names and values will be written."
    
    But I can't really know whether that's the case because it's really confusing.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146712534
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def buildContainerInfo(conf: SparkConf):
    +    ContainerInfo.Builder = {
    --- End diff --
    
    ok


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r144681733
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -170,9 +175,119 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
           containerInfo.addNetworkInfos(info)
         }
     
    +    getSecretVolume(conf, secretConfig).foreach { volume =>
    +      if (volume.getSource.getSecret.getReference.isInitialized) {
    +        logInfo(s"Setting reference secret ${volume.getSource.getSecret.getReference.getName}" +
    +          s"on file ${volume.getContainerPath}")
    +      } else {
    +        logInfo(s"Setting secret on file name=${volume.getContainerPath}")
    +      }
    +      containerInfo.addVolumes(volume)
    +    }
    +
         containerInfo
       }
     
    +  def addSecretEnvVar(
    --- End diff --
    
    Is it possible to make this return `List[Variable]` like it used to as opposed to mutating the `Environment.Builder`, just more consistent (e.g. `getSecretVolume`)


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146680574
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -501,23 +503,78 @@ See the [configuration page](configuration.html) for information on Spark config
         spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
         written to the provided file. Paths are relative to the container's work
         directory.  Absolute paths must already exist.  Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Note: File-based secrets require a custom
    +    <a href="http://mesos.apache.org/documentation/latest/secrets/">SecretResolver
    +    module</a>. Example:
    +    
    +    <pre>filename1,filename2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.names</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret references. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretname1,secretname2</pre>
       </td>
     </tr>
     <tr>
       <td><code>spark.mesos.driver.secret.values</code></td>
       <td><code>(none)</code></td>
       <td>
         A comma-separated list of secret values. Consult the Mesos Secret
    -    protobuf for more information.
    +    protobuf for more information. Example:
    +    
    +    <pre>secretvalue1,secretvalue2</pre>
    +  </td>
    +</tr>
    +
    +<tr>
    +  <td><code>spark.mesos.executor.secret.envkeys</code></td>
    +  <td><code>(none)</code></td>
    +  <td>
    +    A comma-separated list that, if set, the contents of the secret referenced
    +    by spark.mesos.executor.secret.names or spark.mesos.executor.secret.values will be
    +    set to the provided environment variable in the executor's process. Example:
    +    
    +    <pre>ENVKEY1,ENVKEY2</pre>
    +  </td>
    +  </tr>
    +  <tr>
    +<td><code>spark.mesos.executor.secret.filenames</code></td>
    --- End diff --
    
    Indentation is off.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r146987092
  
    --- Diff: docs/running-on-mesos.md ---
    @@ -485,39 +485,87 @@ See the [configuration page](configuration.html) for information on Spark config
     </tr>
     
     <tr>
    -  <td><code>spark.mesos.driver.secret.envkeys</code></td>
    -  <td><code>(none)</code></td>
       <td>
    -    A comma-separated list that, if set, the contents of the secret referenced
    -    by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be
    -    set to the provided environment variable in the driver's process.
    +    <code>spark.mesos.driver.secret.values</code>,
    --- End diff --
    
    Did you generate the docs to see how this looks? I wonder how browsers will try to render this extra-wide column.


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143546897
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -17,10 +17,14 @@
     
     package org.apache.spark.scheduler.cluster.mesos
     
    -import org.apache.mesos.Protos.{ContainerInfo, Image, NetworkInfo, Parameter, Volume}
    +import org.apache.mesos.Protos._
    --- End diff --
    
    Hmm, IntelliJ added this automatically - I'll fix it.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82565 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82565/testReport)** for PR 19437 at commit [`73b2cbf`](https://github.com/apache/spark/commit/73b2cbfeb43a3448a570c055d62c439e267490be).


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437
  
    **[Test build #82867 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82867/testReport)** for PR 19437 at commit [`770d307`](https://github.com/apache/spark/commit/770d307bd67796b93f20d7dba105905059af7a0e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

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


---

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


[GitHub] spark pull request #19437: [SPARK-22131][MESOS] Mesos driver secrets

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

    https://github.com/apache/spark/pull/19437#discussion_r143548722
  
    --- Diff: resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala ---
    @@ -122,7 +126,7 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
         .toList
       }
     
    -  def containerInfo(conf: SparkConf): ContainerInfo.Builder = {
    +  def containerInfo(conf: SparkConf, secretConfig: MesosSecretConfig): ContainerInfo.Builder = {
    --- End diff --
    
    OK, I'll use `buildContainerInfo`.


---

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