You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by liyinan926 <gi...@git.apache.org> on 2018/01/04 06:33:15 UTC

[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

GitHub user liyinan926 opened a pull request:

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

    [SPARK-22953][K8S] Avoids adding duplicated secret volumes when init-container is used

    ## What changes were proposed in this pull request?
    
    User-specified secrets are mounted into both the main container and init-container (when it is used) in a Spark driver/executor pod, using the `MountSecretsBootstrap`. Because `MountSecretsBootstrap` always adds new secret volumes for the secrets to the pod, the same secret volumes get added twice, one when mounting the secrets to the main container, and the other when mounting the secrets to the init-container. This PR fixes the issue by introducing a boolean flag to `MountSecretsBootstrap.mountSecrets` that tells whether to add new secret volumes. The flag is set to `true` when mounting secrets to the main container and `false` when mounting to the init-container.
    
    Ref: https://github.com/apache-spark-on-k8s/spark/issues/594. 
    
    ## How was this patch tested?
    Unit tested.
    
    @vanzin @jiangxb1987 @ueshin This is to fix a critical bug in 2.3. PTAL. Thanks!
    @hex108 @foxish @kimoonkim
    


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

    $ git pull https://github.com/liyinan926/spark-k8s branch-2.3

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

    https://github.com/apache/spark/pull/20148.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 #20148
    
----
commit 9be26a857164e815594941adff6a8d2001d69001
Author: Yinan Li <li...@...>
Date:   2018-01-04T06:20:01Z

    [SPARK-22953][K8S] Avoids adding duplicated secret volumes when init-container is used

----


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159588348
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -28,20 +28,26 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String,
        *
        * @param pod the pod into which the secret volumes are being added.
        * @param container the container into which the secret volumes are being mounted.
    +   * @param addNewVolumes whether to add new secret volumes for the secrets.
    --- End diff --
    
    I think this problem arose because we're conflating two things here - adding secret volumes (which are pod-scoped) and adding volume-mounts (which are container-scoped). I think we should separate these out. The branching may work for now, but we should have a future work item to separate these out.
    
    cc/ @mccheah 


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159739212
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala ---
    @@ -119,7 +119,7 @@ private[spark] class BasicDriverConfigurationStep(
             .endEnv()
           .addNewEnv()
             .withName(ENV_DRIVER_ARGS)
    -        .withValue(appArgs.map(arg => "\"" + arg + "\"").mkString(" "))
    +        .withValue(appArgs.mkString(" "))
    --- End diff --
    
    A note on this change: it seems double quotes get literally added. With the double quotes appended explicitly here, when running SparkPi with an argument 5, an NumberFormatException will be thrown complaining that `""5""` is not parsable. So the explicit double quotes are not really needed.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85693 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85693/testReport)** for PR 20148 at commit [`6636410`](https://github.com/apache/spark/commit/6636410df0007a2877991d90253d096e0667e60f).
     * This patch **fails Spark unit 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159740606
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala ---
    @@ -119,7 +119,7 @@ private[spark] class BasicDriverConfigurationStep(
             .endEnv()
           .addNewEnv()
             .withName(ENV_DRIVER_ARGS)
    -        .withValue(appArgs.map(arg => "\"" + arg + "\"").mkString(" "))
    +        .withValue(appArgs.mkString(" "))
    --- End diff --
    
    Thanks for calling this out - let's try and test this using an integration test as well - https://github.com/apache-spark-on-k8s/spark-integration/. Would be good to put in manual integration testing results here until we have the PRB automation figured out.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85668 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85668/testReport)** for PR 20148 at commit [`92251aa`](https://github.com/apache/spark/commit/92251aa5d518083ae4016f307b48a31d67b87067).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85667 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85667/testReport)** for PR 20148 at commit [`2e6f7db`](https://github.com/apache/spark/commit/2e6f7db4efd9b819eee4c5d6d0152638ba0931d1).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85669 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85669/testReport)** for PR 20148 at commit [`93e1d64`](https://github.com/apache/spark/commit/93e1d648d56079dcc8bd28bafa7dad4cd9d2dfeb).


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Actually no. Please open the PR against master instead.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85690 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85690/testReport)** for PR 20148 at commit [`b17fe4d`](https://github.com/apache/spark/commit/b17fe4db54a409b6fc4b5f8cd0382f5db2924f95).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    LGTM, please also update the PR description.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    @jiangxb1987 Updated PR description.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85695 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85695/testReport)** for PR 20148 at commit [`5278ea7`](https://github.com/apache/spark/commit/5278ea760d0896616b275a1f1f0ac9a7b11ec616).


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85695/
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159588871
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -28,20 +28,26 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String,
        *
        * @param pod the pod into which the secret volumes are being added.
        * @param container the container into which the secret volumes are being mounted.
    +   * @param addNewVolumes whether to add new secret volumes for the secrets.
        * @return the updated pod and container with the secrets mounted.
        */
    -  def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +  def mountSecrets(
    --- End diff --
    
    Can we separate this method into `addSecretVolumes` and `mountSecrets`? The former would just need the pod argument, and the latter would take the container as argument. That way, we could probably separate this out better and it would make it more readable than the branching. WDYT?


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85695 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85695/testReport)** for PR 20148 at commit [`5278ea7`](https://github.com/apache/spark/commit/5278ea760d0896616b275a1f1f0ac9a7b11ec616).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85669/
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159729720
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SecretVolumeUtils.scala ---
    @@ -14,23 +14,20 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.spark.deploy.k8s.submit
    +package org.apache.spark.deploy.k8s
     
     import scala.collection.JavaConverters._
     
     import io.fabric8.kubernetes.api.model.{Container, Pod}
     
     private[spark] object SecretVolumeUtils {
     
    -  def podHasVolume(driverPod: Pod, volumeName: String): Boolean = {
    -    driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    +  def podHasVolume(pod: Pod, volumeName: String): Boolean = {
    +    pod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
       }
     
    -  def containerHasVolume(
    -      driverContainer: Container,
    -      volumeName: String,
    -      mountPath: String): Boolean = {
    -    driverContainer.getVolumeMounts.asScala.exists(volumeMount =>
    +  def containerHasVolume(container: Container, volumeName: String, mountPath: String): Boolean = {
    +    container.getVolumeMounts.asScala.exists(volumeMount =>
    --- 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159590150
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -28,20 +28,26 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String,
        *
        * @param pod the pod into which the secret volumes are being added.
        * @param container the container into which the secret volumes are being mounted.
    +   * @param addNewVolumes whether to add new secret volumes for the secrets.
        * @return the updated pod and container with the secrets mounted.
        */
    -  def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +  def mountSecrets(
    --- End diff --
    
    Good point. Done.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85664 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85664/testReport)** for PR 20148 at commit [`9be26a8`](https://github.com/apache/spark/commit/9be26a857164e815594941adff6a8d2001d69001).


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159729690
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SecretVolumeUtils.scala ---
    @@ -14,23 +14,20 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.spark.deploy.k8s.submit
    +package org.apache.spark.deploy.k8s
     
     import scala.collection.JavaConverters._
     
     import io.fabric8.kubernetes.api.model.{Container, Pod}
     
     private[spark] object SecretVolumeUtils {
     
    -  def podHasVolume(driverPod: Pod, volumeName: String): Boolean = {
    -    driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    +  def podHasVolume(pod: Pod, volumeName: String): Boolean = {
    +    pod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    --- End diff --
    
    Done.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Got it. @vanzin created https://github.com/apache/spark/pull/20159.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85668 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85668/testReport)** for PR 20148 at commit [`92251aa`](https://github.com/apache/spark/commit/92251aa5d518083ae4016f307b48a31d67b87067).


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85664 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85664/testReport)** for PR 20148 at commit [`9be26a8`](https://github.com/apache/spark/commit/9be26a857164e815594941adff6a8d2001d69001).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85664/
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85694 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85694/testReport)** for PR 20148 at commit [`92ef568`](https://github.com/apache/spark/commit/92ef568b388ab3465732fe9c8c01d4aec0e1a40e).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159751143
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala ---
    @@ -119,7 +119,7 @@ private[spark] class BasicDriverConfigurationStep(
             .endEnv()
           .addNewEnv()
             .withName(ENV_DRIVER_ARGS)
    -        .withValue(appArgs.map(arg => "\"" + arg + "\"").mkString(" "))
    +        .withValue(appArgs.mkString(" "))
    --- End diff --
    
    Sure, I will add one to the PR under review.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85669 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85669/testReport)** for PR 20148 at commit [`93e1d64`](https://github.com/apache/spark/commit/93e1d648d56079dcc8bd28bafa7dad4cd9d2dfeb).
     * 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159729003
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SecretVolumeUtils.scala ---
    @@ -14,23 +14,20 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.spark.deploy.k8s.submit
    +package org.apache.spark.deploy.k8s
     
     import scala.collection.JavaConverters._
     
     import io.fabric8.kubernetes.api.model.{Container, Pod}
     
     private[spark] object SecretVolumeUtils {
     
    -  def podHasVolume(driverPod: Pod, volumeName: String): Boolean = {
    -    driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    +  def podHasVolume(pod: Pod, volumeName: String): Boolean = {
    +    pod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    --- End diff --
    
    Since you're touching this, the style is always `.exists { foo => ... }`.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    @vanzin Can this be merged?


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    **[Test build #85694 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85694/testReport)** for PR 20148 at commit [`92ef568`](https://github.com/apache/spark/commit/92ef568b388ab3465732fe9c8c01d4aec0e1a40e).


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Why master?


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

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


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159592335
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala ---
    @@ -172,10 +172,8 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
           "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]())
     
         assert(executor.getSpec.getInitContainers.size() === 1)
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName
    -      === "secret1-volume")
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0)
    -      .getMountPath === "/var/secret1")
    +    assert(SecretVolumeUtils.containerHasVolume(
    --- End diff --
    
    Done. https://github.com/apache/spark/pull/20148/commits/93e1d648d56079dcc8bd28bafa7dad4cd9d2dfeb.


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159591760
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala ---
    @@ -172,10 +172,8 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
           "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]())
     
         assert(executor.getSpec.getInitContainers.size() === 1)
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName
    -      === "secret1-volume")
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0)
    -      .getMountPath === "/var/secret1")
    +    assert(SecretVolumeUtils.containerHasVolume(
    --- End diff --
    
    Thanks!  We also need check volumes' num in pod spec.
    
    +  // check volume mounted.
    + assert(executor.getSpec.getVolumes.size() === 1)
    + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1")


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159588468
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -28,20 +28,26 @@ private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String,
        *
        * @param pod the pod into which the secret volumes are being added.
        * @param container the container into which the secret volumes are being mounted.
    +   * @param addNewVolumes whether to add new secret volumes for the secrets.
    --- End diff --
    
    Agreed. I didn't separate it out because we will touch this code as part of refactoring the steps code anyway.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #20148: [SPARK-22953][K8S] Avoids adding duplicated secret volum...

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

    https://github.com/apache/spark/pull/20148
  
    Because all changes go to master first.


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159590343
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala ---
    @@ -172,10 +172,8 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
           "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]())
     
         assert(executor.getSpec.getInitContainers.size() === 1)
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName
    -      === "secret1-volume")
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0)
    -      .getMountPath === "/var/secret1")
    +    assert(SecretVolumeUtils.containerHasVolume(
    --- End diff --
    
    It might be better to change `None` at line 168 to `Some(secretBootstrap)` and check volumes' number to avoid regression.  


---

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


[GitHub] spark pull request #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159590877
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala ---
    @@ -172,10 +172,8 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
           "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]())
     
         assert(executor.getSpec.getInitContainers.size() === 1)
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName
    -      === "secret1-volume")
    -    assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0)
    -      .getMountPath === "/var/secret1")
    +    assert(SecretVolumeUtils.containerHasVolume(
    --- 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 #20148: [SPARK-22953][K8S] Avoids adding duplicated secre...

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

    https://github.com/apache/spark/pull/20148#discussion_r159729042
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SecretVolumeUtils.scala ---
    @@ -14,23 +14,20 @@
      * See the License for the specific language governing permissions and
      * limitations under the License.
      */
    -package org.apache.spark.deploy.k8s.submit
    +package org.apache.spark.deploy.k8s
     
     import scala.collection.JavaConverters._
     
     import io.fabric8.kubernetes.api.model.{Container, Pod}
     
     private[spark] object SecretVolumeUtils {
     
    -  def podHasVolume(driverPod: Pod, volumeName: String): Boolean = {
    -    driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
    +  def podHasVolume(pod: Pod, volumeName: String): Boolean = {
    +    pod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName)
       }
     
    -  def containerHasVolume(
    -      driverContainer: Container,
    -      volumeName: String,
    -      mountPath: String): Boolean = {
    -    driverContainer.getVolumeMounts.asScala.exists(volumeMount =>
    +  def containerHasVolume(container: Container, volumeName: String, mountPath: String): Boolean = {
    +    container.getVolumeMounts.asScala.exists(volumeMount =>
    --- End diff --
    
    Same here.


---

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