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 2017/12/12 20:18:47 UTC

[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] add init-container boot...

GitHub user liyinan926 opened a pull request:

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

    [SPARK-22757][Kubernetes] add init-container bootstrapping and secret mounting steps

    ## What changes were proposed in this pull request?
    
    This PR expands the Kubernetes mode to be able to use remote dependencies on http/https endpoints, GCS, S3, etc. It adds steps for configuring and appending the Kubernetes init-container into the driver and executor pods for downloading remote dependencies. 
    [Init-containers](https://kubernetes.io/docs/concepts/workloads/pods/init-containers/), as the name suggests, are containers that are run to completion before the main containers start, and are often used to perform initialization tasks prior to starting the main containers. We use init-containers to localize remote application dependencies before the driver/executors start running. The code that the init-container runs is also included. This PR also adds a step to the driver and executors for mounting user-specified secrets that may store credentials for accessing data storage, e.g., S3 and Google Cloud Storage (GCS), into the driver and executors. 
    
    ## How was this patch tested?
    
    * The patch contains unit tests which are passing.
    * Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
    * It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
    * There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI.
    * Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html
    
    cc @rxin @felixcheung @mateiz (shepherd)
    k8s-big-data SIG members & contributors: @mccheah @foxish @ash211 @ssuchter @varunkatta @kimoonkim @erikerlandson @tnachen @ifilonenko @liyinan926
    reviewers: @vanzin @felixcheung @jiangxb1987 @mridulm

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

    $ git pull https://github.com/liyinan926/spark-k8s init-container

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

    https://github.com/apache/spark/pull/19954.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 #19954
    
----
commit 8aa875906ad11780587543f87634c21d3ca94677
Author: Yinan Li <li...@gmail.com>
Date:   2017-12-12T18:24:27Z

    [SPARK-22757][Kubernetes] add init-container bootstrapping and secret mounting steps

----


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85013 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85013/testReport)** for PR 19954 at commit [`197882d`](https://github.com/apache/spark/commit/197882d787b602b80b0a9b500133463a95a9a691).
     * 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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157631357
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    --- End diff --
    
    What's the purpose of all these traits that have a single implementation? That seems unnecessary.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin Addressed your comments in https://github.com/apache/spark/pull/19954/commits/3407d7af68c44b100558e49e4012a27e41b29dda. PTAL. Thanks a lot! 


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Jenkins, retest this please.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157891637
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.{File, FileInputStream}
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SparkConfPropertiesParser {
    +
    +  def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
    +    val sparkConf = new SparkConf(true)
    +
    +    if (!propertiesFile.isFile) {
    +      throw new IllegalArgumentException("Server properties file given at " +
    +        s"${propertiesFile.getAbsoluteFile} does not exist or is not a file.")
    +    }
    +
    +    val properties = new Properties
    --- End diff --
    
    There's probably already a method or a combination of calls in the `Utils` class that does what you want. e.g. `loadDefaultSparkProperties` or `getPropertiesFromFile` or some other.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > It's nice that there is some documentation somewhere, but that documentation doesn't really seem to address my comments. For one example, it only explicitly talks about the driver - which sort of makes sense because the document is about submission. But why aren't orchestrators used when starting executors too? It seems there's similar code baked into another class instead.
    
    > What I'm asking is for this to be documented properly so that someone who didn't write the code has enough information to know that it's working as it should. Right now I don't see what some of these abstractions are for at all - for example, as far as I can see, the orchestrator can be replaced by a method call instead of being a completely separate type; it's not really abstracting anything. Look at where it's used:
    
    @vanzin Thanks for this feedback. I might suggest that documentation isn't the main issue, but rather that it's possible the abstractions themselves are not self-documenting. We're open to ideas for alternative representations of the submission logic that are easier to parse for the outside reader, and this dialogue is very productive.
    
    > But why aren't orchestrators used when starting executors too? It seems there's similar code baked into another class instead.
    
    As with any abstraction, we can inline the orchestrator into the submission client, and then consider if that makes the `Client` do too many things, and also, what would the unit tests look like as a result? I like that we can test the orchestrator and its selection of which steps to apply independently from testing that the `Client` takes the API objects and creates them accordingly, particularly with the hooked up owner references. A goal we had with this design is that every class has as few responsibilities as possible; ideally every module is responsible for exactly one thing. Though the long constructor argument list for the orchestrator would suggest that the orchestrator is tied to the Client pretty tightly.
    
    > But why aren't orchestrators used when starting executors too? It seems there's similar code baked into another class instead.
    
    A common situation one will see is that the submission client needs to do strictly more work than the driver does, as there is more work required to set up a Spark application and create the objects for it, than it is to bootstrap the executor pods with the objects that already exist. For example, the submission client has to create the config map that sets up the init-container, as well as mount the config map volume into the driver; but the driver does not create the config map but instead uses the pre-existing one, but still has to do the same mounting into the executor pod as the submission client does. These semantics are shown where we first have a submodule that attaches the config map volumes to an arbitrary pod (the “bootstrap”), then, the driver need only use that submodule in isolation, but the submission client wraps that submodule in a step that both uses that submodule and also creates the API objects in the API server.
    
    It would be neat to explore the idea of using a step-based system that is shared for creating the executors and creating the driver, but we have to think carefully about a proper abstraction. For example, the `KubernetesDriverSpec` has the notion of `additionalKubernetesResources` and the `driverSparkConf`, which are constructs that do not apply when creating the executor pods. This illustrates what we observe above - the construction of the application is strictly more work than creating the executors. So then, does the submission client have two orchestrators - one for configuring the driver pod itself, and one for configuring everything else in the application, e.g. creating config maps? Then the orchestration and step selection for the pod-configuration-steps could be shared between the submission client and the driver, but the orchestrator for strictly creating Kubernetes API objects and the driver spark conf would only be used by the submission client. There are some direct
 ions we can explore here, and we are open to ideas.
    
    > So aside from not really being able to infer the structure of how these things work, the current abstraction seems to be creating a lot of constructors and methods with long lists of arguments, which is another thing that hurts the readability of the code.
    
    I think regardless of the abstractions we choose, we're eventually going to end up with many classes with unique constructors simply due to the complexity of the submission client itself. Creating a Spark application in Kubernetes has many steps, and we want to put each step into a submodule - regardless if the submodules have a shared trait like we have now, or if the submodules are just independent classes with independent APIs. Each submodule will have an argument list that is befitting of the properties that submodule will require to configure the driver. See https://github.com/apache-spark-on-k8s/spark/tree/branch-2.2-kubernetes/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps for all the different submodules the submission client will eventually need.
    
    I do think that we should remove the `InitContainer` level of steps and orchestration, as having two layers of steps and orchestration is pretty confusing. There’s only one code branch in the init container steps orchestrator which we only apply when we are also submitting local files - and that isn’t even done in this PR - so it shouldn’t be too big of a burden to inline that logic all into the top level step when the time comes.
    
    Again, the feedback is very much appreciated. What are some ways we can improve and move forward here?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157445117
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.{File, FileInputStream}
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SparkConfPropertiesParser {
    +
    +  def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
    +    val sparkConf = new SparkConf(true)
    +
    +    if (!propertiesFile.isFile) {
    +      throw new IllegalArgumentException(s"Server properties file given at" +
    --- End diff --
    
    nit: remove `s`.
    nit: move a space at the beginning of the string of the next line to the end of the string of this line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167643
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.MountSecretsBootstrap
    +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils}
    +
    +class DriverMountSecretsStepSuite extends SparkFunSuite {
    +
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/driver"
    +
    +  test("Mounts all given secrets") {
    +    val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false))
    +    val secretNamesToMountPaths = Map(
    +      SECRET_FOO -> SECRET_MOUNT_PATH,
    +      SECRET_BAR -> SECRET_MOUNT_PATH)
    +
    +    val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths)
    +    val mountSecretsStep = new DriverMountSecretsStep(mountSecretsBootstrap)
    +    val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec)
    +    val driverPodWithSecretsMounted = configuredDriverSpec.driverPod
    +    val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer
    +
    +    Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName =>
    +      assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName)))
    +    Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName =>
    --- End diff --
    
    .foreach { volumeName =>
    



---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157883847
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) {
    +
    +  /**
    +   * Mounts Kubernetes secrets as secret volumes into the given container in the given pod.
    +   *
    +   * @param pod the pod into which the secret volumes are being added.
    +   * @param container the container into which the secret volumes are being mounted.
    +   * @return the updated pod and container with the secrets mounted.
    +   */
    +  def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +    var podBuilder = new PodBuilder(pod)
    +    secretNamesToMountPaths.keys.foreach { name =>
    +      podBuilder = podBuilder
    +        .editOrNewSpec()
    +          .addNewVolume()
    +          .withName(secretVolumeName(name))
    +          .withNewSecret()
    +            .withSecretName(name)
    +            .endSecret()
    +          .endVolume()
    +          .endSpec()
    +    }
    +
    +    var containerBuilder = new ContainerBuilder(container)
    +    secretNamesToMountPaths.foreach { namePath =>
    --- End diff --
    
    Use `case (foo, bar)` to avoid `_1` and `_2` below.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84838/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157914419
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    --- End diff --
    
    Done. Added `spark.kubernetes.initContainer.maxThreadPoolSize` with a default value of 5.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327224
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    --- End diff --
    
    Renamed to `SparkPodInitContainer`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157909168
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.{File, FileInputStream}
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SparkConfPropertiesParser {
    +
    +  def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
    +    val sparkConf = new SparkConf(true)
    +
    +    if (!propertiesFile.isFile) {
    +      throw new IllegalArgumentException("Server properties file given at " +
    +        s"${propertiesFile.getAbsoluteFile} does not exist or is not a file.")
    +    }
    +
    +    val properties = new Properties
    --- End diff --
    
    Yes, `Utils.loadDefaultSparkProperties` works.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84966 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84966/testReport)** for PR 19954 at commit [`38b850f`](https://github.com/apache/spark/commit/38b850ffa4e1cb188e8e7614109bf811f3777998).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157912841
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Figures out and returns the complete ordered list of InitContainerConfigurationSteps required to
    + * configure the driver init-container. The returned steps will be applied in the given order to
    + * produce a final InitContainerSpec that is used to construct the driver init-container in
    + * DriverInitContainerBootstrapStep. This class is only used when an init-container is needed, i.e.,
    + * when there are remote application dependencies to localize.
    + */
    +private[spark] class InitContainerConfigOrchestrator(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    imagePullPolicy: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkConf: SparkConf) {
    +
    +  private val initContainerImage = sparkConf
    +    .get(INIT_CONTAINER_IMAGE)
    +    .getOrElse(throw new SparkException(
    +      "Must specify the init-container image when there are remote dependencies"))
    +
    +  def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = {
    +    val initContainerBootstrap = new InitContainerBootstrap(
    +      initContainerImage,
    +      imagePullPolicy,
    +      jarsDownloadPath,
    +      filesDownloadPath,
    +      configMapName,
    +      configMapKey,
    +      SPARK_POD_DRIVER_ROLE,
    +      sparkConf)
    +    val baseStep = new BaseInitContainerConfigurationStep(
    +      sparkJars,
    +      sparkFiles,
    +      jarsDownloadPath,
    +      filesDownloadPath,
    +      initContainerBootstrap)
    +
    +    val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_DRIVER_SECRETS_PREFIX)
    +    // Mount user-specified driver secrets also into the driver's init-container. The
    +    // init-container may need credentials in the secrets to be able to download remote
    +    // dependencies. The driver's main container and its init-container share the secrets
    +    // because the init-container is sort of an implementation details and this sharing
    +    // avoids introducing a dedicated configuration property just for the init-container.
    +    val maybeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158166901
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE)
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    --- End diff --
    
    Fits in previous line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85398 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85398/testReport)** for PR 19954 at commit [`28343fb`](https://github.com/apache/spark/commit/28343fb50310826bc9962e785f25d1af9b3c3f4a).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911819
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    if (filesCommaSeparated.isDefined) {
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157320353
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] trait MountSecretsBootstrap {
    +
    +  /**
    +   * Mounts Kubernetes secrets as secret volumes into the given container in the given pod.
    +   *
    +   * @param pod the pod into which the secret volumes are being added.
    +   * @param container the container into which the secret volumes are being mounted.
    +   * @return the updated pod and container with the secrets mounted.
    +   */
    +  def mountSecrets(pod: Pod, container: Container): (Pod, Container)
    +}
    +
    +private[spark] class MountSecretsBootstrapImpl(
    +    secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap {
    +
    +  override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +    var podBuilder = new PodBuilder(pod)
    +    secretNamesToMountPaths.keys.foreach(name =>
    +      podBuilder = podBuilder
    +        .editOrNewSpec()
    +          .addNewVolume()
    +          .withName(secretVolumeName(name))
    +          .withNewSecret()
    +            .withSecretName(name)
    +            .endSecret()
    +          .endVolume()
    +          .endSpec())
    +
    +    var containerBuilder = new ContainerBuilder(container)
    +    secretNamesToMountPaths.foreach(namePath =>
    --- End diff --
    
    `.foreach { namePath =>`. You get the idea. Please fix all of these.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651951
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (initContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157442475
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala ---
    @@ -72,9 +76,57 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
         )
       }
     
    +  test("Submission steps with an init-container.") {
    +    val sparkConf = new SparkConf(false)
    +      .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE)
    +      .set(INIT_CONTAINER_IMAGE, IC_IMAGE)
    +      .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar")
    +    val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
    +    val orchestrator = new DriverConfigOrchestrator(
    +      NAMESPACE,
    +      APP_ID,
    +      LAUNCH_TIME,
    +      Some(mainAppResource),
    +      APP_NAME,
    +      MAIN_CLASS,
    +      APP_ARGS,
    +      sparkConf)
    +    validateStepTypes(
    +      orchestrator,
    +      classOf[BaseDriverConfigurationStep],
    +      classOf[DriverServiceBootstrapStep],
    +      classOf[DriverKubernetesCredentialsStep],
    +      classOf[DependencyResolutionStep],
    +      classOf[DriverInitContainerBootstrapStep])
    +  }
    +
    +  test("Submission steps with driver secrets to mount") {
    +    val sparkConf = new SparkConf(false)
    +      .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE)
    +      .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH)
    +      .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH)
    +    val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
    +    val orchestrator = new DriverConfigOrchestrator(
    +      NAMESPACE,
    +      APP_ID,
    +      LAUNCH_TIME,
    +      Some(mainAppResource),
    +      APP_NAME,
    +      MAIN_CLASS,
    +      APP_ARGS,
    +      sparkConf)
    +    validateStepTypes(
    +      orchestrator,
    +      classOf[BaseDriverConfigurationStep],
    +      classOf[DriverServiceBootstrapStep],
    +      classOf[DriverKubernetesCredentialsStep],
    +      classOf[DependencyResolutionStep],
    +      classOf[DriverMountSecretsStep])
    +  }
    +
       private def validateStepTypes(
    -      orchestrator: DriverConfigurationStepsOrchestrator,
    -      types: Class[_ <: DriverConfigurationStep]*): Unit = {
    +                                 orchestrator: DriverConfigOrchestrator,
    +                                 types: Class[_ <: DriverConfigurationStep]*): Unit = {
    --- End diff --
    
    nit: indent


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158680033
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile ---
    @@ -0,0 +1,24 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +FROM spark-base
    +
    +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
    +# command should be invoked from the top level directory of the Spark distribution. E.g.:
    +# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile .
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651386
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE)
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize))
    +
    +  private val jarsDownloadDir = new File(sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    --- End diff --
    
    nit: `errMessageOnDestinationNotADirectory` -> `errMessage`?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157885683
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala ---
    @@ -116,10 +127,45 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    +    val mayBeInitContainerBootstrapStep =
    +      if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +        val orchestrator = new InitContainerConfigOrchestrator(
    +          sparkJars,
    +          sparkFiles,
    +          jarsDownloadPath,
    +          filesDownloadPath,
    +          imagePullPolicy,
    +          initContainerConfigMapName,
    +          INIT_CONTAINER_PROPERTIES_FILE_NAME,
    +          sparkConf)
    +        val bootstrapStep = new DriverInitContainerBootstrapStep(
    +          orchestrator.getAllConfigurationSteps,
    +          initContainerConfigMapName,
    +          INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +
    +        Some(bootstrapStep)
    +      } else {
    +        None
    +      }
    +
    +    val mayBeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    --- End diff --
    
    ```
    val mountSecretsSteps = if (...) Seq(...) else Nil
    ```
    
    You could probably use the same logic above too for the init container. Using an option here is not really doing anything useful.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85056/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323581
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Returns the complete ordered list of steps required to configure the init-container. This is
    + * only used when there are remote application dependencies to localize.
    + */
    +private[spark] class InitContainerConfigurationStepsOrchestrator(
    --- End diff --
    
    Similarly, some type names are also really long. `InitContainerOrchestrator` sounds just as good to me.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157913710
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    I'm not saying that they should be doing the same work, I'm saying that because they are not using the same abstraction, you can't use the same existing concepts here. You had to create the concept of a "bootstrap" to reuse code between the "driver orchestrator" and this code (the "executor orchestrator"), and kinda shoerhorn it into the whole thing so that you could share a little bit of code.
    
    Whereas if both could reuse steps (like "initialize this init container"), you probably could avoid that altogether.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84966/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157815526
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +private[spark] class BaseInitContainerConfigurationStep(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    bootstrap: InitContainerBootstrap)
    +  extends InitContainerConfigurationStep {
    +
    +  override def configureInitContainer(spec: InitContainerSpec): InitContainerSpec = {
    +    val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
    +    val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
    +    val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
    +    } else {
    +      Map()
    +    }
    +    val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(","))
    +    } else {
    +      Map()
    +    }
    +
    +    val baseInitContainerConfig = Map(
    +      JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath,
    +      FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++
    +      remoteJarsConf ++
    +      remoteFilesConf
    +    val bootstrapped = bootstrap.bootstrapInitContainer(
    +      PodWithDetachedInitContainer(
    +        spec.driverPod,
    +        spec.initContainer,
    +        spec.driverContainer))
    +
    +    spec.copy(
    +      initContainer = bootstrapped.initContainer,
    +      driverContainer = bootstrapped.mainContainer,
    +      driverPod = bootstrapped.pod,
    +      properties = baseInitContainerConfig)
    --- End diff --
    
    Yes, good point.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157141640
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val initContainerCustomEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
    +      .toSeq
    +      .map(env =>
    +        new EnvVarBuilder()
    +          .withName(env._1)
    +          .withValue(env._2)
    +          .build())
    +
    +    val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer)
    +      .withName(s"spark-init")
    +      .withImage(initContainerImage)
    +      .withImagePullPolicy(dockerImagePullPolicy)
    +      .addAllToEnv(initContainerCustomEnvVars.asJava)
    +      .addNewVolumeMount()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
    +        .endVolumeMount()
    +      .addToVolumeMounts(sharedVolumeMounts: _*)
    +      .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
    +      .build()
    +
    +    val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod)
    +      .editSpec()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withNewConfigMap()
    +          .withName(initContainerConfigMapName)
    +          .addNewItem()
    +            .withKey(initContainerConfigMapKey)
    +            .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +            .endItem()
    +          .endConfigMap()
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .endSpec()
    +      .build()
    +
    +    val mainContainerWithMountedFiles = new ContainerBuilder(
    +      podWithDetachedInitContainer.mainContainer)
    +        .addToVolumeMounts(sharedVolumeMounts: _*)
    +        .addNewEnv()
    +          .withName(ENV_MOUNTED_FILES_DIR)
    +          .withValue(filesDownloadPath)
    +          .endEnv()
    +        .build()
    +
    +    PodWithDetachedInitContainer(
    +      podWithBasicVolumes,
    +      initContainer,
    +      mainContainerWithMountedFiles)
    +  }
    +
    --- End diff --
    
    nit: remove an extra line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157641069
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    --- End diff --
    
    It's more idiomatic to mock a `trait` than a `class` and our unit tests always create mocks for every component that isn't the class under test.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85122 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85122/testReport)** for PR 19954 at commit [`429146d`](https://github.com/apache/spark/commit/429146db5ce3986fceef4b6b3381c7bac2fbd97d).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > We discuss this a bit more in here
    
    It's nice that there is some documentation somewhere, but that documentation doesn't really seem to address my comments. For one example, it only explicitly talks about the driver - which sort of makes sense because the document is about submission. But why aren't orchestrators used when starting executors too? It seems there's similar code baked into another class instead.
    
    What I'm asking is for this to be documented properly so that someone who didn't write the code has enough information to know that it's working as it should. Right now I don't see what some of these abstractions are for at all - for example, as far as I can see, the orchestrator can be replaced by a method call instead of being a completely separate type; it's not really abstracting anything. Look at where it's used:
    
    ```
        val configurationStepsOrchestrator = new DriverConfigOrchestrator(...)
    
        Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient(/* another long list of arguments */)0) { kubernetesClient =>
            val client = new Client(
              configurationStepsOrchestrator.getAllConfigurationSteps(),
              /* another long list of arguments */
    ```
    
    So aside from not really being able to infer the structure of how these things work, the current abstraction seems to be creating a lot of constructors and methods with long lists of arguments, which is another thing that hurts the readability of the code.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158165025
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,63 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) {
    +
    +  /**
    +   * Mounts Kubernetes secrets as secret volumes into the given container in the given pod.
    +   *
    +   * @param pod the pod into which the secret volumes are being added.
    +   * @param container the container into which the secret volumes are being mounted.
    +   * @return the updated pod and container with the secrets mounted.
    +   */
    +  def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +    var podBuilder = new PodBuilder(pod)
    +    secretNamesToMountPaths.keys.foreach { name =>
    +      podBuilder = podBuilder
    +        .editOrNewSpec()
    +          .addNewVolume()
    +          .withName(secretVolumeName(name))
    +          .withNewSecret()
    +            .withSecretName(name)
    +            .endSecret()
    +          .endVolume()
    +          .endSpec()
    +    }
    +
    +    var containerBuilder = new ContainerBuilder(container)
    +    secretNamesToMountPaths.foreach {
    +      case (name, path) =>
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85398/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157246571
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          s"or is not a directory.")
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157246489
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,60 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (maybeInitContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (maybeInitContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    // Only set up the bootstrap if they've provided both the config map key and the config map
    +    // name. The config map might not be provided if init-containers aren't being used to
    +    // bootstrap dependencies.
    +    val maybeInitContainerBootstrap = for {
    +      configMap <- maybeInitContainerConfigMap
    +      configMapKey <- maybeInitContainerConfigMapKey
    +    } yield {
    +      val initContainerImage = sparkConf
    +        .get(INIT_CONTAINER_DOCKER_IMAGE)
    +        .getOrElse(throw new SparkException(
    +          "Must specify the init-container Docker image when there are remote dependencies"))
    +      new InitContainerBootstrapImpl(
    +        initContainerImage,
    +        sparkConf.get(DOCKER_IMAGE_PULL_POLICY),
    +        sparkConf.get(JARS_DOWNLOAD_LOCATION),
    +        sparkConf.get(FILES_DOWNLOAD_LOCATION),
    +        sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT),
    +        configMap,
    +        configMapKey,
    +        SPARK_POD_EXECUTOR_ROLE,
    +        sparkConf)
    +    }
    +
    +    val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX)
    +    val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) {
    +      Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths))
    +    } else {
    +      None
    +    }
    +    // Mount user-specified executor secrets also into the executor's init-container. The
    +    // init-container may need credentials in the secrets to be able to download remote
    +    // dependencies. The executor's main container and its init-container share the secrets
    +    // because the init-container is sort of an implementation details and this sharing
    +    // avoids introducing a dedicated configuration property just for the init-container.
    +    val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty &&
    --- End diff --
    
    `mayBeMountSecretBootstrap` is for the executor main container, whereas `mayBeInitContainerMountSecretsBootstrap` is for the executor init container, which is run before the main container starts. 


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157246586
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +214,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map {bootstrap =>
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85029 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85029/testReport)** for PR 19954 at commit [`340fa41`](https://github.com/apache/spark/commit/340fa4107cabdf325ec884855098e4ded3c666fa).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323760
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala ---
    @@ -0,0 +1,27 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +
    +/**
    + * Utility for fetching remote file dependencies.
    + */
    +private[spark] trait FileFetcher {
    --- End diff --
    
    Why do you need a trait for this? If it's for the tests, you can mock classes, you don't need an interface for that.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157892401
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    if (filesCommaSeparated.isDefined) {
    +      require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory)
    +    }
    +    filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file =>
    --- End diff --
    
    Use `Utils.stringToSeq`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157889714
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +/**
    + * Performs basic configuration for the driver init-container with most of the work delegated to
    + * the given InitContainerBootstrap.
    + */
    +private[spark] class BaseInitContainerConfigurationStep(
    --- End diff --
    
    It's kinda weird for a class called `BaseBlah` to extend a class called `Blah`. Perhaps `Basic` instead.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    +1 to @foxish comment above.  For accessing data from http, s3, etc this will be a huge reduction in barrier to entry. The difference between having to spin a custom docker image and just using it out of the box.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84985 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84985/testReport)** for PR 19954 at commit [`46a8c99`](https://github.com/apache/spark/commit/46a8c9961312ee820743ddf893cc8666ce9360fa).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157893811
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (maybeInitContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (maybeInitContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    // Only set up the bootstrap if they've provided both the config map key and the config map
    +    // name. The config map might not be provided if init-containers aren't being used to
    +    // bootstrap dependencies.
    +    val maybeInitContainerBootstrap = for {
    --- End diff --
    
    Optimally, you (the orchestrator?) would just instantiate all the steps and each step would be responsible for figuring out if it has anything to do... that would simplify code in the call sites (like here).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > I disagree. For example, just to pick one example I spotted while glancing at the code quickly. There are many steps that take a kubernetesResourceNamePrefix as a constructor argument. Then the code that instantiates those steps needs to know each of those steps, and that they take that prefix as an argument to their constructor. I don't see that as abstracting much.
    
    > Instead, for example, if you encapsulate the state of a "kubernetes submission" into some helper type that wraps SparkConf and other common things needed to submit an app, those steps could potentially just take that helper type as an argument to the abstract method that does the processing (e.g. configureDriver). There's less step-specific logic needed everywhere, and the list of arguments to constructors and methods shrinks a lot.
    
    @vanzin An excellent idea - this is very much in line with the e.g. `SQLConf` that's in Spark-SQL. @liyinan926 it would be great to introduce this and swap out our parameters in various places. Our original methodology was to pass parameters down that would normally need to be repeatedly retrieved by the same config key from the `SparkConf` over and over again, but we can make an abstraction for that. This basically moves us from parameter primitives to parameter objects, an elegant solution.
    
    > That's the thing. That's not an abstraction, that's just code that lives in a separate method. You can test the "method that creates a list of steps" separately too without having to stash it in a completely separate type and create more coupling in your code. Abstraction is meant to reduce coupling, not increase it ...
    
    > ... You just have code living in separate classes with the goal of making testing easier, but you can achieve the same thing without the code living in separate classes, and things would probably be simpler overall.
    
    A design we aimed for here was to make every class in the code do exactly one thing, and the test for that class thus can focus solely on testing that single thing. Having the `@VisibleForTesting` annotation with public methods or something similar to test multiple methods in the same class makes that single unit test class difficult to read. Or we would have to split up the unit test for the given class to multiple unit test classes, but then it becomes difficult to track down all of the classes that come together to test that class. So although I agree that we're not creating abstractions per se in the sense that we're not creating generic APIs that have multiple implementations, we are creating helper objects and submodules to decompose the submission process such that we have smaller chunks to test at a time.
    
    Therefore a key hallmark for why the code is decomposed the way it is, is to make the tests as readable as possible, that each test class is honed in on covering all code paths in a single unit. Again - open to ideas of how to make that more readable in the main code.
    
    > An abstraction for an orchestrator would mean that there is a code path that can take a different orchestrator to achieve different things. For example, if you had code that started containers, and given a different orchestrator, could start either a driver or an executor, or even be used by tests to mock some behavior you want to verify.
    
    I'm interested in exploring this idea - will have to think a bit more on what that would specifically look like.
    
    > I think the biggest thing that makes reviewing this PR challenging is the way the code is structured, not the amount of changes. This PR is actually both shorter and conceptually simpler than the first two big ones, IMO. Unless we do a thorough refactoring of the code, I personally don't think splitting into smaller ones will help much on reducing the confusion. Splitting also risks making half baked feature into 2.3, which I personally don't think is a good idea.
    
    @liyinan926 the code structure is the main issue, yes, but we can tackle the code structure more effectively by having a better decomposition of the review process as well. We have these three distinct components which are relatively independent. We can therefore separate out the three pieces and consider the architecture for each of them individually. Whereas here we may want to tackle how to change the submission and the executor pod configuration, but what if in our focus on those we miss crucial issues in the design of the init container? Breaking up the PR will allow us to make each PR focus on the architectural considerations of each part more thoroughly, and it makes it easier to follow the discussion surrounding each part.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157709460
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +private[spark] class BaseInitContainerConfigurationStep(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    bootstrap: InitContainerBootstrap)
    +  extends InitContainerConfigurationStep {
    +
    +  override def configureInitContainer(spec: InitContainerSpec): InitContainerSpec = {
    +    val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
    +    val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
    +    val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
    +    } else {
    +      Map()
    +    }
    +    val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(","))
    +    } else {
    +      Map()
    +    }
    +
    +    val baseInitContainerConfig = Map(
    +      JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath,
    +      FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++
    +      remoteJarsConf ++
    +      remoteFilesConf
    +    val bootstrapped = bootstrap.bootstrapInitContainer(
    +      PodWithDetachedInitContainer(
    +        spec.driverPod,
    +        spec.initContainer,
    +        spec.driverContainer))
    +
    +    spec.copy(
    +      initContainer = bootstrapped.initContainer,
    +      driverContainer = bootstrapped.mainContainer,
    +      driverPod = bootstrapped.pod,
    +      properties = baseInitContainerConfig)
    --- End diff --
    
    `properties = spec.properties ++ baseInitContainerConfig`, just in case?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167023
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE)
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    filesCommaSeparated.foreach { files =>
    +      require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory)
    +      Utils.stringToSeq(files).foreach { file =>
    +        Future[Unit] {
    +          fileFetcher.fetchFile(file, downloadDir)
    +        }
    +
    --- End diff --
    
    nuke empty line.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911827
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    if (filesCommaSeparated.isDefined) {
    +      require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory)
    +    }
    +    filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file =>
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157320177
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val initContainerCustomEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
    +      .toSeq
    +      .map(env =>
    --- End diff --
    
    `.map { env =>`


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158168099
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.UUID
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import org.mockito.Mockito
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mockito.MockitoSugar._
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.util.Utils
    +
    +class SparkPodInitContainerSuite
    +  extends SparkFunSuite with BeforeAndAfter {
    --- End diff --
    
    Fits in 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167923
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.MountSecretsBootstrap
    +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils}
    +
    +class DriverMountSecretsStepSuite extends SparkFunSuite {
    +
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/driver"
    +
    +  test("Mounts all given secrets") {
    --- End diff --
    
    start test names with lower case


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84789 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84789/testReport)** for PR 19954 at commit [`f9dc86d`](https://github.com/apache/spark/commit/f9dc86d3a0af85027d24b76dc50849da70f4053e).
     * This patch **fails Scala style 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 #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84798 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84798/testReport)** for PR 19954 at commit [`5512d80`](https://github.com/apache/spark/commit/5512d80ef0d293ededc0a57f3d34c70e69e1bc30).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @mridulm, can you PTAL? This is ready for another round of review, and is the last large feature we're planning to do in 2.3.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651933
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    +      .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " +
    +        "locations into the driver and executor pods.")
    +      .timeConf(TimeUnit.MINUTES)
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651930
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala ---
    @@ -98,28 +109,62 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    -    val sparkJars = submissionSparkConf.getOption("spark.jars")
    +    val sparkJars = sparkConf.getOption("spark.jars")
           .map(_.split(","))
           .getOrElse(Array.empty[String]) ++
           additionalMainAppJar.toSeq
    -    val sparkFiles = submissionSparkConf.getOption("spark.files")
    +    val sparkFiles = sparkConf.getOption("spark.files")
           .map(_.split(","))
           .getOrElse(Array.empty[String])
     
    -    val maybeDependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) {
    -      Some(new DependencyResolutionStep(
    +    val dependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) {
    +      Seq(new DependencyResolutionStep(
             sparkJars,
             sparkFiles,
             jarsDownloadPath,
             filesDownloadPath))
         } else {
    -      None
    +      Nil
    +    }
    +
    +    val initContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +      val orchestrator = new InitContainerConfigOrchestrator(
    +        sparkJars,
    +        sparkFiles,
    +        jarsDownloadPath,
    +        filesDownloadPath,
    +        imagePullPolicy,
    +        initContainerConfigMapName,
    +        INIT_CONTAINER_PROPERTIES_FILE_NAME,
    +        sparkConf)
    +      val bootstrapStep = new DriverInitContainerBootstrapStep(
    +        orchestrator.getAllConfigurationSteps,
    +        initContainerConfigMapName,
    +        INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +
    +      Seq(bootstrapStep)
    +    } else {
    +      Nil
    +    }
    +
    +    val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    +      Seq(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths)))
    +    } else {
    +      Nil
         }
     
         Seq(
           initialSubmissionStep,
    -      driverAddressStep,
    +      serviceBootstrapStep,
           kubernetesCredentialsStep) ++
    -      maybeDependencyResolutionStep.toSeq
    +      dependencyResolutionStep ++
    +      initContainerBootstrapStep ++
    +      mountSecretsStep
    +  }
    +
    +  private def areAnyFilesNonContainerLocal(files: Seq[String]): 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157531197
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringReader
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +import org.apache.spark.util.Utils
    +
    +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite {
    +
    +  private val CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val CONFIG_MAP_KEY = "spark-init-config-map-key"
    +
    +  test("The init container bootstrap step should use all of the init container steps") {
    +    val baseDriverSpec = KubernetesDriverSpec(
    +      driverPod = new PodBuilder().build(),
    +      driverContainer = new ContainerBuilder().build(),
    +      driverSparkConf = new SparkConf(false),
    +      otherKubernetesResources = Seq.empty[HasMetadata])
    +    val initContainerSteps = Seq(
    +      FirstTestInitContainerConfigurationStep$,
    +      SecondTestInitContainerConfigurationStep$)
    +    val bootstrapStep = new DriverInitContainerBootstrapStep(
    +      initContainerSteps,
    +      CONFIG_MAP_NAME,
    +      CONFIG_MAP_KEY)
    +
    +    val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec)
    +
    +    assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala ===
    +      FirstTestInitContainerConfigurationStep$.additionalLabels)
    +    val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
    +    assert(additionalDriverEnv.size === 1)
    +    assert(additionalDriverEnv.head.getName ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey)
    +    assert(additionalDriverEnv.head.getValue ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue)
    +
    +    assert(preparedDriverSpec.otherKubernetesResources.size === 2)
    +    assert(preparedDriverSpec.otherKubernetesResources.contains(
    +      FirstTestInitContainerConfigurationStep$.additionalKubernetesResource))
    +    assert(preparedDriverSpec.otherKubernetesResources.exists {
    +      case configMap: ConfigMap =>
    +        val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME
    +        val configMapData = configMap.getData.asScala
    +        val hasCorrectNumberOfEntries = configMapData.size == 1
    +        val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY)
    +        val initContainerProperties = new Properties()
    +        Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) {
    +          initContainerProperties.load(_)
    +        }
    +        val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala
    +        val expectedInitContainerProperties = Map(
    +          SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey ->
    +            SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue)
    +        val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties
    +        hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties
    +      case _ => false
    +    })
    +
    +    val initContainers = preparedDriverSpec.driverPod.getSpec.getInitContainers
    +    assert(initContainers.size() === 1)
    +    val initContainerEnv = initContainers.get(0).getEnv.asScala
    +    assert(initContainerEnv.size === 1)
    +    assert(initContainerEnv.head.getName ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey)
    +    assert(initContainerEnv.head.getValue ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue)
    +
    +    val expectedSparkConf = Map(
    +      INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME,
    +      INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY,
    +      SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey ->
    +        SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue)
    +    assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf)
    +  }
    +}
    +
    +private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
    --- End diff --
    
    No, it shouldn't be there. Removed.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158165498
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala ---
    @@ -31,21 +32,23 @@ private[spark] class DependencyResolutionStep(
         sparkJars: Seq[String],
         sparkFiles: Seq[String],
         jarsDownloadPath: String,
    -    localFilesDownloadPath: String) extends DriverConfigurationStep {
    +    filesDownloadPath: String) extends DriverConfigurationStep {
     
       override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
    -    val resolvedSparkJars = KubernetesFileUtils.resolveFileUris(sparkJars, jarsDownloadPath)
    -    val resolvedSparkFiles = KubernetesFileUtils.resolveFileUris(
    -      sparkFiles, localFilesDownloadPath)
    -    val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone()
    +    val resolvedSparkJars = KubernetesUtils.resolveFileUris(sparkJars, jarsDownloadPath)
    +    val resolvedSparkFiles = KubernetesUtils.resolveFileUris(
    +      sparkFiles, filesDownloadPath)
    --- End diff --
    
    Fits in previous line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157914224
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    The difference in the work involved is the reason why we need different abstractions between the two places. What's a cleaner abstraction that allows for this difference in workload?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157880769
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Bootstraps an init-container for downloading remote dependencies. This is separated out from
    + * the init-container steps API because this component can be used to bootstrap init-containers
    + * for both the driver and executors.
    + */
    +private[spark] class InitContainerBootstrap(
    +    initContainerImage: String,
    +    imagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf) {
    +
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val customEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env =>
    --- End diff --
    
    s/`env =>`/`case (key, value) =>`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85327/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158168037
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +class InitContainerConfigOrchestratorSuite extends SparkFunSuite {
    +
    +  private val DOCKER_IMAGE = "init-container"
    +  private val SPARK_JARS = Seq(
    +    "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
    +  private val SPARK_FILES = Seq(
    +    "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
    +  private val JARS_DOWNLOAD_PATH = "/var/data/jars"
    +  private val FILES_DOWNLOAD_PATH = "/var/data/files"
    +  private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent"
    +  private val CUSTOM_LABEL_KEY = "customLabel"
    +  private val CUSTOM_LABEL_VALUE = "customLabelValue"
    +  private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key"
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/init-container"
    +
    +  test ("including basic configuration step") {
    +    val sparkConf = new SparkConf(true)
    +      .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE)
    +      .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE)
    +
    +    val orchestrator = new InitContainerConfigOrchestrator(
    +      SPARK_JARS.take(1),
    +      SPARK_FILES,
    +      JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_PATH,
    +      DOCKER_IMAGE_PULL_POLICY,
    +      INIT_CONTAINER_CONFIG_MAP_NAME,
    +      INIT_CONTAINER_CONFIG_MAP_KEY,
    +      sparkConf)
    +    val initSteps : Seq[InitContainerConfigurationStep] =
    +      orchestrator.getAllConfigurationSteps
    +    assert(initSteps.lengthCompare(1) == 0)
    +    assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep])
    +  }
    +
    +  test("including step to mount user-specified secrets") {
    +    val sparkConf = new SparkConf(false)
    +      .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE)
    +      .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH)
    +      .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH)
    +
    +    val orchestrator = new InitContainerConfigOrchestrator(
    +      SPARK_JARS.take(1),
    +      SPARK_FILES,
    +      JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_PATH,
    +      DOCKER_IMAGE_PULL_POLICY,
    +      INIT_CONTAINER_CONFIG_MAP_NAME,
    +      INIT_CONTAINER_CONFIG_MAP_KEY,
    +      sparkConf)
    +    val initSteps : Seq[InitContainerConfigurationStep] =
    --- End diff --
    
    no space before `:`; in fact the type isn't really needed here.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85324 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85324/testReport)** for PR 19954 at commit [`785b90e`](https://github.com/apache/spark/commit/785b90e52580e9175896b22b00b23f30fbe020ef).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84985 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84985/testReport)** for PR 19954 at commit [`46a8c99`](https://github.com/apache/spark/commit/46a8c9961312ee820743ddf893cc8666ce9360fa).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    @liyinan926, can you update the PR title please?
    Maybe "Enable submission of GCS (or S3) and HTTP dependencies to Kubernetes Scheduler Backend"


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85069 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85069/testReport)** for PR 19954 at commit [`ddcb0f2`](https://github.com/apache/spark/commit/ddcb0f2c988557b73a389e781133aa4621d3bb4f).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157914870
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    I've already made a ton of suggestions. I'm not asking you to modify this PR, I'm pointing out places where having a proper abstraction would clean up the code.
    
    e.g. this is the place where calling a common "prepare this container" code taking a custom orchestrator for executors would help.
    
    Otherwise, if tomorrow you need to add another step before starting driver and executors, you'll need another hack like "bootstraps" because they don't share the concept of an orchestrator that runs config steps.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157322696
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala ---
    @@ -47,6 +47,18 @@ private[spark] object KubernetesFileUtils {
         }
       }
     
    +  /**
    +   * Get from a given collection of file URIs the ones that represent remote files.
    +   */
    +  def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = {
    +    filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local")
    +  }
    +
    +  private def filterUriStringsByScheme(
    +      uris: Iterable[String], schemeFilter: (String => Boolean)): Iterable[String] = {
    --- End diff --
    
    One param per line for multi-line signatures. But why not inline this since it's only called in one place?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    +1. This PR closes a big feature gap of the Kubernetes scheduler backend compared with other backends. This makes the Kubernetes backend much more usable. 


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911785
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +/**
    + * Performs basic configuration for the driver init-container with most of the work delegated to
    + * the given InitContainerBootstrap.
    + */
    +private[spark] class BaseInitContainerConfigurationStep(
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

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


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157913485
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (maybeInitContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (maybeInitContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    // Only set up the bootstrap if they've provided both the config map key and the config map
    +    // name. The config map might not be provided if init-containers aren't being used to
    +    // bootstrap dependencies.
    +    val maybeInitContainerBootstrap = for {
    --- End diff --
    
    Agreed. This part definitely needs refactoring once we start using an orchestrator pattern for the executors.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @ueshin addressed your comments. PTAL. Thanks!


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157324021
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    --- End diff --
    
    This name is so long it becomes confusing. How about `KubernetesInitContainer`? Or do you plan to have multiple different init containers for different things (ugh)?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157916165
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    We can address the refactors in a follow-up - it would be good to get a refactor going pretty soon though since a lot of the future work will depend on the fundamental architecture here.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84789/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911709
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Bootstraps an init-container for downloading remote dependencies. This is separated out from
    + * the init-container steps API because this component can be used to bootstrap init-containers
    + * for both the driver and executors.
    + */
    +private[spark] class InitContainerBootstrap(
    +    initContainerImage: String,
    +    imagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf) {
    +
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val customEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env =>
    +      new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +    }
    +
    +    val initContainer = new ContainerBuilder(original.initContainer)
    +      .withName("spark-init")
    +      .withImage(initContainerImage)
    +      .withImagePullPolicy(imagePullPolicy)
    +      .addAllToEnv(customEnvVars.asJava)
    +      .addNewVolumeMount()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
    +        .endVolumeMount()
    +      .addToVolumeMounts(sharedVolumeMounts: _*)
    +      .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
    +      .build()
    +
    +    val podWithBasicVolumes = new PodBuilder(original.pod)
    +      .editSpec()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withNewConfigMap()
    +          .withName(configMapName)
    +          .addNewItem()
    +            .withKey(configMapKey)
    +            .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +            .endItem()
    +          .endConfigMap()
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .endSpec()
    +      .build()
    +
    +    val mainContainer = new ContainerBuilder(
    +      original.mainContainer)
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157159984
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,60 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (maybeInitContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (maybeInitContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    // Only set up the bootstrap if they've provided both the config map key and the config map
    +    // name. The config map might not be provided if init-containers aren't being used to
    +    // bootstrap dependencies.
    +    val maybeInitContainerBootstrap = for {
    +      configMap <- maybeInitContainerConfigMap
    +      configMapKey <- maybeInitContainerConfigMapKey
    +    } yield {
    +      val initContainerImage = sparkConf
    +        .get(INIT_CONTAINER_DOCKER_IMAGE)
    +        .getOrElse(throw new SparkException(
    +          "Must specify the init-container Docker image when there are remote dependencies"))
    +      new InitContainerBootstrapImpl(
    +        initContainerImage,
    +        sparkConf.get(DOCKER_IMAGE_PULL_POLICY),
    +        sparkConf.get(JARS_DOWNLOAD_LOCATION),
    +        sparkConf.get(FILES_DOWNLOAD_LOCATION),
    +        sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT),
    +        configMap,
    +        configMapKey,
    +        SPARK_POD_EXECUTOR_ROLE,
    +        sparkConf)
    +    }
    +
    +    val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX)
    +    val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) {
    +      Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths))
    +    } else {
    +      None
    +    }
    +    // Mount user-specified executor secrets also into the executor's init-container. The
    +    // init-container may need credentials in the secrets to be able to download remote
    +    // dependencies. The executor's main container and its init-container share the secrets
    +    // because the init-container is sort of an implementation details and this sharing
    +    // avoids introducing a dedicated configuration property just for the init-container.
    +    val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty &&
    --- End diff --
    
    What's the difference between this and `mayBeMountSecretBootstrap` above? Looks like mounting the same paths twice?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157892194
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    --- End diff --
    
    Probably good to put a configurable upper limit to the number of threads here, since the tasks they're running are probably not super short-lived.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911768
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala ---
    @@ -0,0 +1,37 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit
    +
    +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder}
    +
    +private[spark] object InitContainerUtil {
    +
    +  /**
    +   * Append (add to the list of InitContainers) a given init-container to a pod.
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85056 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85056/testReport)** for PR 19954 at commit [`340fa41`](https://github.com/apache/spark/commit/340fa4107cabdf325ec884855098e4ded3c666fa).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85105 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85105/testReport)** for PR 19954 at commit [`5b82fc0`](https://github.com/apache/spark/commit/5b82fc06a5fff2a3ff2062c46a21d069640eeec7).
     * 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 #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85327 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85327/testReport)** for PR 19954 at commit [`f82c568`](https://github.com/apache/spark/commit/f82c5682565bc0e1bc34ec428faedb53ee5ddecd).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85066 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85066/testReport)** for PR 19954 at commit [`9ebfc73`](https://github.com/apache/spark/commit/9ebfc7325be8713e41c7684d182d2c69d173b811).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85029 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85029/testReport)** for PR 19954 at commit [`340fa41`](https://github.com/apache/spark/commit/340fa4107cabdf325ec884855098e4ded3c666fa).
     * This patch **fails SparkR 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 #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    I'll finish reading this by Friday, thanks!


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > I don't think they are independent as architecturally they make sense together and represent a single concern: enabling use of remote dependencies through init-containers. Missing any one of the three makes the feature unusable. I would also argue that it won't necessarily make review easier as reviewers need to mentally connect them together to make sense of each change set. 
    
    I agree with this. This is pretty much one cohesive unit and splitting it up is going to probably lead to more difficulty in understanding it. From your comments @vanzin, it seems we definitely do need a good refactor here, and the community can undertake that in Q1 2018. This approach and code has been functionally tested over the last 3 releases of our fork - and I'd be fairly confident about its efficacy - broad changes at this point seem riskier to me from a 2.3 release perspective given that we're still in the process of improving spark-k8s integration testing coverage against apache/spark. 
    
    cc/ @mccheah 


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157443182
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringReader
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +import org.apache.spark.util.Utils
    +
    +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite {
    +
    +  private val CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val CONFIG_MAP_KEY = "spark-init-config-map-key"
    +
    +  test("The init container bootstrap step should use all of the init container steps") {
    +    val baseDriverSpec = KubernetesDriverSpec(
    +      driverPod = new PodBuilder().build(),
    +      driverContainer = new ContainerBuilder().build(),
    +      driverSparkConf = new SparkConf(false),
    +      otherKubernetesResources = Seq.empty[HasMetadata])
    +    val initContainerSteps = Seq(
    +      FirstTestInitContainerConfigurationStep$,
    +      SecondTestInitContainerConfigurationStep$)
    +    val bootstrapStep = new DriverInitContainerBootstrapStep(
    +      initContainerSteps,
    +      CONFIG_MAP_NAME,
    +      CONFIG_MAP_KEY)
    +
    +    val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec)
    +
    +    assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala ===
    +      FirstTestInitContainerConfigurationStep$.additionalLabels)
    +    val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
    +    assert(additionalDriverEnv.size === 1)
    +    assert(additionalDriverEnv.head.getName ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey)
    +    assert(additionalDriverEnv.head.getValue ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue)
    +
    +    assert(preparedDriverSpec.otherKubernetesResources.size === 2)
    +    assert(preparedDriverSpec.otherKubernetesResources.contains(
    +      FirstTestInitContainerConfigurationStep$.additionalKubernetesResource))
    +    assert(preparedDriverSpec.otherKubernetesResources.exists {
    +      case configMap: ConfigMap =>
    +        val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME
    +        val configMapData = configMap.getData.asScala
    +        val hasCorrectNumberOfEntries = configMapData.size == 1
    +        val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY)
    +        val initContainerProperties = new Properties()
    +        Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) {
    +          initContainerProperties.load(_)
    +        }
    +        val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala
    +        val expectedInitContainerProperties = Map(
    +          SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey ->
    +            SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue)
    +        val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties
    +        hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties
    +      case _ => false
    +    })
    +
    +    val initContainers = preparedDriverSpec.driverPod.getSpec.getInitContainers
    +    assert(initContainers.size() === 1)
    +    val initContainerEnv = initContainers.get(0).getEnv.asScala
    +    assert(initContainerEnv.size === 1)
    +    assert(initContainerEnv.head.getName ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey)
    +    assert(initContainerEnv.head.getValue ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue)
    +
    +    val expectedSparkConf = Map(
    +      INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME,
    +      INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY,
    +      SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey ->
    +        SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue)
    +    assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf)
    +  }
    +}
    +
    +private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
    --- End diff --
    
    Do we need `$` at the end of the object name?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    This is promising ! I will take look at it over the weekend, thanks for the great work :-)


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651915
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    --- End diff --
    
    Please see the response regarding `spark.kubernetes.mountDependencies.maxSimultaneousDownloads`.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > I also don't fully understand all the abstractions being created here. It seems there are three separate concepts (a "bootstrap", a "configuration step", and an "orchestrator") and they're not used consistently.
    
    A configuration step is a logical unit that applies an additive transformation to the input. A steps orchestrator selects which configuration steps to apply based on the configuration of the application. A bootstrap is a component that can be shared by one or more configuration steps and the driver, since a lot of times the submission client and the executor will share code for configuring the driver and the executor pods, respectively. We discuss this a bit more in here: https://github.com/apache-spark-on-k8s/spark/blob/branch-2.2-kubernetes/resource-managers/kubernetes/architecture-docs/submission-client.md - but we don't cover the bootstrap abstraction. We're open to different representations and architectures as well.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin Addressed your comments so far. PTAL. Thanks!


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651945
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE)
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize))
    +
    +  private val jarsDownloadDir = new File(sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85066/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323338
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,63 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +private[spark] class BaseInitContainerConfigurationStep(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    initContainerBootstrap: InitContainerBootstrap)
    +  extends InitContainerConfigurationStep {
    +
    +  override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
    +    val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
    +    val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
    +    val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
    +    } else {
    +      Map.empty[String, String]
    +    }
    +    val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(","))
    +    } else {
    +      Map.empty[String, String]
    --- End diff --
    
    `Map()`.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin @mccheah @foxish Variables names have been shortened in https://github.com/apache/spark/pull/19954/commits/d50c61e46c43049b9ede615ffabd89a55c4adb0c, and traits with only a single implementation have been removed in https://github.com/apache/spark/pull/19954/commits/5b82fc06a5fff2a3ff2062c46a21d069640eeec7. Regarding the documentation of the concepts around steps and orchestrators and how they fit together, I would suggest that we have a separate PR for the architecture docs.  I also think we can refactor the orchestrator logic in a future PR to also cover executor pods so we use the same mechanism for both the driver and executor pods. WDYT?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158675633
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile ---
    @@ -0,0 +1,24 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +FROM spark-base
    +
    +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
    +# command should be invoked from the top level directory of the Spark distribution. E.g.:
    +# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile .
    --- End diff --
    
    `kubernetes/dockerfiles/..` instead of `dockerfiles/..`
    
    Btw, only nits but seems like paths here in `Dockerfile`s for driver/executor are wrong: `kubernetes/dockerfiles/driver/Dockerfile` and `kubernetes/dockerfiles/executor/Dockerfile` respectively?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85352 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85352/testReport)** for PR 19954 at commit [`9d9c841`](https://github.com/apache/spark/commit/9d9c841b3528e0806280a58a0a8acaa456aa6e44).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157913013
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    The submission client has to do more work in each step than the driver has to do to configure executors. The submission client's steps also set up accompanying Kubernetes resources and the driver's spark configuration, whereas the driver only has to use those Kubernetes resources to mount volumes into the pods.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157888663
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala ---
    @@ -0,0 +1,80 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrap, MountSecretsBootstrap}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Figures out and returns the complete ordered list of InitContainerConfigurationSteps required to
    + * configure the driver init-container. The returned steps will be applied in the given order to
    + * produce a final InitContainerSpec that is used to construct the driver init-container in
    + * DriverInitContainerBootstrapStep. This class is only used when an init-container is needed, i.e.,
    + * when there are remote application dependencies to localize.
    + */
    +private[spark] class InitContainerConfigOrchestrator(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    imagePullPolicy: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkConf: SparkConf) {
    +
    +  private val initContainerImage = sparkConf
    +    .get(INIT_CONTAINER_IMAGE)
    +    .getOrElse(throw new SparkException(
    +      "Must specify the init-container image when there are remote dependencies"))
    +
    +  def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = {
    +    val initContainerBootstrap = new InitContainerBootstrap(
    +      initContainerImage,
    +      imagePullPolicy,
    +      jarsDownloadPath,
    +      filesDownloadPath,
    +      configMapName,
    +      configMapKey,
    +      SPARK_POD_DRIVER_ROLE,
    +      sparkConf)
    +    val baseStep = new BaseInitContainerConfigurationStep(
    +      sparkJars,
    +      sparkFiles,
    +      jarsDownloadPath,
    +      filesDownloadPath,
    +      initContainerBootstrap)
    +
    +    val secretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_DRIVER_SECRETS_PREFIX)
    +    // Mount user-specified driver secrets also into the driver's init-container. The
    +    // init-container may need credentials in the secrets to be able to download remote
    +    // dependencies. The driver's main container and its init-container share the secrets
    +    // because the init-container is sort of an implementation details and this sharing
    +    // avoids introducing a dedicated configuration property just for the init-container.
    +    val maybeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    --- End diff --
    
    Same about using `Seq` directly instead of an option.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85142 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85142/testReport)** for PR 19954 at commit [`3407d7a`](https://github.com/apache/spark/commit/3407d7af68c44b100558e49e4012a27e41b29dda).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911840
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157321761
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala ---
    @@ -116,10 +122,53 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    +    val mayBeInitContainerBootstrapStep =
    +      if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +        val initContainerConfigurationStepsOrchestrator =
    --- End diff --
    
    The variable names in this code are very long in general; verbosity can both help and harm readability, in this case I don't think it's helping much. For example, `orchestrator` is just as good a name for this variable, since there's no other orchestrator being used here.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85142 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85142/testReport)** for PR 19954 at commit [`3407d7a`](https://github.com/apache/spark/commit/3407d7af68c44b100558e49e4012a27e41b29dda).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Thanks! merging to master.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @liyinan926 we might be able to make the review smaller, though this makes some of the code we contribute on each push unusable on its own. If we're ok with that, how about we create these commits:
    
    - Init container by itself that downloads the files
    - Submission client changes to attach the init container to the driver
    - Driver changes to attach the init container to the executors
    
    Thoughts?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85126 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85126/testReport)** for PR 19954 at commit [`20fff03`](https://github.com/apache/spark/commit/20fff03655bd73785a235d88236ea4c3978ae7ec).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157141827
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val initContainerCustomEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
    +      .toSeq
    +      .map(env =>
    +        new EnvVarBuilder()
    +          .withName(env._1)
    +          .withValue(env._2)
    +          .build())
    +
    +    val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer)
    +      .withName(s"spark-init")
    --- End diff --
    
    nit: remove `s`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327721
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -133,30 +132,78 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.docker.image")
    --- End diff --
    
    >  Is it a required config?
    
    No, as one may forgo the init container if they're building the deps into the docker image itself and supplying it via `local:///` paths.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157893177
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    So this is where my suggestion of getting rid of bootstraps and using inheritance breaks. For that to work, this code would have to use the same abstraction as the driver side (orchestrator + steps), but it doesn't.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323182
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringWriter
    +import java.util.Properties
    +
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata}
    +
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec}
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +
    +/**
    + * Configures the init-container that bootstraps dependencies into the driver pod, including
    + * building a ConfigMap that will be mounted into the init-container. The ConfigMap carries
    + * configuration properties for the init-container.
    + */
    +private[spark] class DriverInitContainerBootstrapStep(
    +    initContainerConfigurationSteps: Seq[InitContainerConfigurationStep],
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String)
    +  extends DriverConfigurationStep {
    +
    +  override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
    +    var currentInitContainerSpec = InitContainerSpec(
    +      initContainerProperties = Map.empty[String, String],
    +      driverSparkConf = Map.empty[String, String],
    +      initContainer = new ContainerBuilder().build(),
    +      driverContainer = driverSpec.driverContainer,
    +      driverPod = driverSpec.driverPod,
    +      initContainerDependentResources = Seq.empty[HasMetadata])
    +    for (nextStep <- initContainerConfigurationSteps) {
    +      currentInitContainerSpec = nextStep.configureInitContainer(currentInitContainerSpec)
    +    }
    +
    +    val configMap = buildConfigMap(
    +      initContainerConfigMapName,
    +      initContainerConfigMapKey,
    +      currentInitContainerSpec.initContainerProperties)
    +    val resolvedDriverSparkConf = driverSpec.driverSparkConf
    +      .clone()
    +      .set(INIT_CONTAINER_CONFIG_MAP_NAME, initContainerConfigMapName)
    +      .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, initContainerConfigMapKey)
    +      .setAll(currentInitContainerSpec.driverSparkConf)
    +    val resolvedDriverPod = InitContainerUtil.appendInitContainer(
    +      currentInitContainerSpec.driverPod, currentInitContainerSpec.initContainer)
    +
    +    driverSpec.copy(
    +      driverPod = resolvedDriverPod,
    +      driverContainer = currentInitContainerSpec.driverContainer,
    +      driverSparkConf = resolvedDriverSparkConf,
    +      otherKubernetesResources =
    +        driverSpec.otherKubernetesResources ++
    +          currentInitContainerSpec.initContainerDependentResources ++
    +          Seq(configMap))
    +  }
    +
    +  private def buildConfigMap(
    +      configMapName: String,
    +      configMapKey: String,
    +      config: Map[String, String]): ConfigMap = {
    +    val properties = new Properties()
    +    config.foreach { entry =>
    +      properties.setProperty(entry._1, entry._2) }
    --- End diff --
    
    `}` goes in next line, or you could have everything in one line.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157898666
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.{File, FileInputStream}
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SparkConfPropertiesParser {
    +
    +  def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
    --- End diff --
    
    Yes, done.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651027
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala ---
    @@ -98,28 +109,62 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    -    val sparkJars = submissionSparkConf.getOption("spark.jars")
    +    val sparkJars = sparkConf.getOption("spark.jars")
           .map(_.split(","))
           .getOrElse(Array.empty[String]) ++
           additionalMainAppJar.toSeq
    -    val sparkFiles = submissionSparkConf.getOption("spark.files")
    +    val sparkFiles = sparkConf.getOption("spark.files")
           .map(_.split(","))
           .getOrElse(Array.empty[String])
     
    -    val maybeDependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) {
    -      Some(new DependencyResolutionStep(
    +    val dependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) {
    +      Seq(new DependencyResolutionStep(
             sparkJars,
             sparkFiles,
             jarsDownloadPath,
             filesDownloadPath))
         } else {
    -      None
    +      Nil
    +    }
    +
    +    val initContainerBootstrapStep = if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +      val orchestrator = new InitContainerConfigOrchestrator(
    +        sparkJars,
    +        sparkFiles,
    +        jarsDownloadPath,
    +        filesDownloadPath,
    +        imagePullPolicy,
    +        initContainerConfigMapName,
    +        INIT_CONTAINER_PROPERTIES_FILE_NAME,
    +        sparkConf)
    +      val bootstrapStep = new DriverInitContainerBootstrapStep(
    +        orchestrator.getAllConfigurationSteps,
    +        initContainerConfigMapName,
    +        INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +
    +      Seq(bootstrapStep)
    +    } else {
    +      Nil
    +    }
    +
    +    val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    +      Seq(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths)))
    +    } else {
    +      Nil
         }
     
         Seq(
           initialSubmissionStep,
    -      driverAddressStep,
    +      serviceBootstrapStep,
           kubernetesCredentialsStep) ++
    -      maybeDependencyResolutionStep.toSeq
    +      dependencyResolutionStep ++
    +      initContainerBootstrapStep ++
    +      mountSecretsStep
    +  }
    +
    +  private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = {
    --- End diff --
    
    nit:`areAnyFilesNonContainerLocal` -> `existNonContainerLocalFiles`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85094 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85094/testReport)** for PR 19954 at commit [`5b82fc0`](https://github.com/apache/spark/commit/5b82fc06a5fff2a3ff2062c46a21d069640eeec7).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911743
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) {
    +
    +  /**
    +   * Mounts Kubernetes secrets as secret volumes into the given container in the given pod.
    +   *
    +   * @param pod the pod into which the secret volumes are being added.
    +   * @param container the container into which the secret volumes are being mounted.
    +   * @return the updated pod and container with the secrets mounted.
    +   */
    +  def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +    var podBuilder = new PodBuilder(pod)
    +    secretNamesToMountPaths.keys.foreach { name =>
    +      podBuilder = podBuilder
    +        .editOrNewSpec()
    +          .addNewVolume()
    +          .withName(secretVolumeName(name))
    +          .withNewSecret()
    +            .withSecretName(name)
    +            .endSecret()
    +          .endVolume()
    +          .endSpec()
    +    }
    +
    +    var containerBuilder = new ContainerBuilder(container)
    +    secretNamesToMountPaths.foreach { namePath =>
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85105 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85105/testReport)** for PR 19954 at commit [`5b82fc0`](https://github.com/apache/spark/commit/5b82fc06a5fff2a3ff2062c46a21d069640eeec7).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84795 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84795/testReport)** for PR 19954 at commit [`b9a0090`](https://github.com/apache/spark/commit/b9a009062dcd56c93190367d35fadb5a00ab1781).
     * This patch **fails to build**.
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323326
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,63 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +private[spark] class BaseInitContainerConfigurationStep(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    initContainerBootstrap: InitContainerBootstrap)
    +  extends InitContainerConfigurationStep {
    +
    +  override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
    +    val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
    +    val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
    +    val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
    +    } else {
    +      Map.empty[String, String]
    --- End diff --
    
    `Map()`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157324170
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    --- End diff --
    
    You have a thread pool, but really you're just submitting two tasks. Why not one task for each file / jar?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157886113
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala ---
    @@ -0,0 +1,37 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit
    +
    +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder}
    +
    +private[spark] object InitContainerUtil {
    +
    +  /**
    +   * Append (add to the list of InitContainers) a given init-container to a pod.
    --- End diff --
    
    "Append the given init-container to a pod's list of init containers."


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85219 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85219/testReport)** for PR 19954 at commit [`be5de00`](https://github.com/apache/spark/commit/be5de00c9a75ef583759b931e3f8597b9f426f6c).
     * This patch **fails PySpark 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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157324262
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    +      remoteJarsDownload,
    +      remoteFilesDownload)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    if (filesCommaSeparated.isDefined) {
    +      require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory)
    +    }
    +    filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file =>
    +      fileFetcher.fetchFile(file, downloadDir)
    +    }
    +  }
    +
    +  private def waitForFutures(futures: Future[_]*) {
    --- End diff --
    
    `: Unit = {`
    
    But really, there's a single caller, so just inline.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84798 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84798/testReport)** for PR 19954 at commit [`5512d80`](https://github.com/apache/spark/commit/5512d80ef0d293ededc0a57f3d34c70e69e1bc30).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157643654
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    --- End diff --
    
    It's probably fine to just use the class here, but some classes can't be mocked, such as final classes or classes with final methods. Having traits everywhere ensures that even if we change the classes down the road to have such characteristics, our tests won't break.
    
    This also is not entirely without precedent. `TaskScheduler` is only implemented by `TaskSchedulerImpl` in the main scheduler code, as is `TaskContext` being extended only by `TaskContextImpl`. Putting a trait in front of an implementation communicates that it's expected for tests that dependency-inject instances of this to create stub implementations.
    
    But we might be splitting hairs at this point, so using only the class could suffice until we run into problems from having done so.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin @jiangxb1987 @ueshin Can this PR be merged now?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157892496
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    logInfo(s"Downloading remote jars: $remoteJars")
    +    downloadFiles(
    +      remoteJars,
    +      jarsDownloadDir,
    +      s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    logInfo(s"Downloading remote files: $remoteFiles")
    +    downloadFiles(
    +      remoteFiles,
    +      filesDownloadDir,
    +      s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +        "or is not a directory.")
    +
    +    downloadExecutor.shutdown()
    +    downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES)
    +  }
    +
    +  private def downloadFiles(
    +      filesCommaSeparated: Option[String],
    +      downloadDir: File,
    +      errMessageOnDestinationNotADirectory: String): Unit = {
    +    if (filesCommaSeparated.isDefined) {
    --- End diff --
    
    Use `filesCommaSeparated.foreach { ...`.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84838 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84838/testReport)** for PR 19954 at commit [`1a74521`](https://github.com/apache/spark/commit/1a74521c3f114a9774598738daef5489c6fa8bae).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157323441
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BaseInitContainerConfigurationStep.scala ---
    @@ -0,0 +1,63 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils
    +
    +private[spark] class BaseInitContainerConfigurationStep(
    +    sparkJars: Seq[String],
    +    sparkFiles: Seq[String],
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    initContainerBootstrap: InitContainerBootstrap)
    +  extends InitContainerConfigurationStep {
    +
    +  override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
    +    val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
    +    val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
    +    val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
    +    } else {
    +      Map.empty[String, String]
    +    }
    +    val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) {
    +      Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(","))
    +    } else {
    +      Map.empty[String, String]
    +    }
    +
    +    val baseInitContainerConfig = Map[String, String](
    --- End diff --
    
    Scala is generally pretty good at inferring types, you don't need to be explicit about the type parameters everywhere.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158650088
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    +      .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " +
    +        "locations into the driver and executor pods.")
    +      .timeConf(TimeUnit.MINUTES)
    --- End diff --
    
    Why not `TimeUnit.SECONDS`?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651523
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (initContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (initContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    --- End diff --
    
    nit: `was not` -> `is not`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @rxin @mateiz @vanzin @mridulm @jiangxb1987 @felixcheung, if you can help take a look at this, it would add a lot of value to Kubernetes mode in 2.3. Granted we already have a way to supply dependencies within docker images and that works as expected (with the minor fix in https://github.com/apache/spark/pull/19972), this would enable submitting from http and from object storage. That's a big value-add for people and enables them to have standard images and not bake a new one for each spark application.
    
    If we can get this review going everyone leaves for vacation (and even if it gets picked up after we all return), that's a big step for us in the community.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85384 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85384/testReport)** for PR 19954 at commit [`c51bc56`](https://github.com/apache/spark/commit/c51bc560bb2ae0d5ea8d914e84d7485d333f497e).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157246540
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val initContainerCustomEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
    +      .toSeq
    +      .map(env =>
    +        new EnvVarBuilder()
    +          .withName(env._1)
    +          .withValue(env._2)
    +          .build())
    +
    +    val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer)
    +      .withName(s"spark-init")
    +      .withImage(initContainerImage)
    +      .withImagePullPolicy(dockerImagePullPolicy)
    +      .addAllToEnv(initContainerCustomEnvVars.asJava)
    +      .addNewVolumeMount()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
    +        .endVolumeMount()
    +      .addToVolumeMounts(sharedVolumeMounts: _*)
    +      .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
    +      .build()
    +
    +    val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod)
    +      .editSpec()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withNewConfigMap()
    +          .withName(initContainerConfigMapName)
    +          .addNewItem()
    +            .withKey(initContainerConfigMapKey)
    +            .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +            .endItem()
    +          .endConfigMap()
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .endSpec()
    +      .build()
    +
    +    val mainContainerWithMountedFiles = new ContainerBuilder(
    +      podWithDetachedInitContainer.mainContainer)
    +        .addToVolumeMounts(sharedVolumeMounts: _*)
    +        .addNewEnv()
    +          .withName(ENV_MOUNTED_FILES_DIR)
    +          .withValue(filesDownloadPath)
    +          .endEnv()
    +        .build()
    +
    +    PodWithDetachedInitContainer(
    +      podWithBasicVolumes,
    +      initContainer,
    +      mainContainerWithMountedFiles)
    +  }
    +
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85126 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85126/testReport)** for PR 19954 at commit [`20fff03`](https://github.com/apache/spark/commit/20fff03655bd73785a235d88236ea4c3978ae7ec).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85122/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167975
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala ---
    @@ -0,0 +1,82 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +class InitContainerConfigOrchestratorSuite extends SparkFunSuite {
    +
    +  private val DOCKER_IMAGE = "init-container"
    +  private val SPARK_JARS = Seq(
    +    "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
    +  private val SPARK_FILES = Seq(
    +    "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
    +  private val JARS_DOWNLOAD_PATH = "/var/data/jars"
    +  private val FILES_DOWNLOAD_PATH = "/var/data/files"
    +  private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent"
    +  private val CUSTOM_LABEL_KEY = "customLabel"
    +  private val CUSTOM_LABEL_VALUE = "customLabelValue"
    +  private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key"
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/init-container"
    +
    +  test ("including basic configuration step") {
    --- End diff --
    
    no space before `(`


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157138621
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +214,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map {bootstrap =>
    --- End diff --
    
    nit: add a space before `bootstrap`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327812
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala ---
    @@ -0,0 +1,27 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +
    +/**
    + * Utility for fetching remote file dependencies.
    + */
    +private[spark] trait FileFetcher {
    --- End diff --
    
    Yeah, removed the trait.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157942418
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) {
    --- End diff --
    
    I created https://issues.apache.org/jira/browse/SPARK-22839 to keep track of the refactoring work.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin Thanks for the review!  Addressed your comments in https://github.com/apache/spark/pull/19954/commits/be5de00c9a75ef583759b931e3f8597b9f426f6c.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84838 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84838/testReport)** for PR 19954 at commit [`1a74521`](https://github.com/apache/spark/commit/1a74521c3f114a9774598738daef5489c6fa8bae).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157631750
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    --- End diff --
    
    I commented about this in my previous review, but could you try to use shorter variable names throughout the PR?
    
    For example, here, just repeating the name of the already long type to name the variable doesn't really help with readability. Imagine if you have two of those, are you going to start adding counters to the already long name?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84966 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84966/testReport)** for PR 19954 at commit [`38b850f`](https://github.com/apache/spark/commit/38b850ffa4e1cb188e8e7614109bf811f3777998).
     * This patch **fails SparkR 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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157328327
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -133,30 +132,78 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.docker.image")
    --- End diff --
    
    Renamed to `spark.kubernetes.initContainer.image`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158650043
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    +      .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " +
    +        "locations into the driver and executor pods.")
    +      .timeConf(TimeUnit.MINUTES)
    +      .createWithDefault(5)
    +
    +  val INIT_CONTAINER_MAX_THREAD_POOL_SIZE =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads")
    --- End diff --
    
    nit: `spark.kubernetes.initContainer.mountDependencies.maxSimultaneousDownloads`?


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167807
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala ---
    @@ -0,0 +1,95 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +import org.mockito.{Mock, MockitoAnnotations}
    +import org.mockito.Matchers.any
    +import org.mockito.Mockito.when
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +
    +class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val SPARK_JARS = Seq(
    +    "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
    +  private val SPARK_FILES = Seq(
    +    "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
    +  private val JARS_DOWNLOAD_PATH = "/var/data/jars"
    +  private val FILES_DOWNLOAD_PATH = "/var/data/files"
    +  private val POD_LABEL = Map("bootstrap" -> "true")
    +  private val INIT_CONTAINER_NAME = "init-container"
    +  private val DRIVER_CONTAINER_NAME = "driver-container"
    +
    +  @Mock
    +  private var podAndInitContainerBootstrap : InitContainerBootstrap = _
    +
    +  before {
    +    MockitoAnnotations.initMocks(this)
    +    when(podAndInitContainerBootstrap.bootstrapInitContainer(
    +      any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] {
    +      override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = {
    +        val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer])
    +        pod.copy(
    +          pod = new PodBuilder(pod.pod)
    +            .withNewMetadata()
    +            .addToLabels("bootstrap", "true")
    +            .endMetadata()
    +            .withNewSpec().endSpec()
    +            .build(),
    +          initContainer = new ContainerBuilder()
    +            .withName(INIT_CONTAINER_NAME)
    +            .build(),
    +          mainContainer = new ContainerBuilder()
    +            .withName(DRIVER_CONTAINER_NAME)
    +            .build()
    +        )}})
    +  }
    +
    +  test("Test of additionalDriverSparkConf with mix of remote files and jars") {
    +    val baseInitStep = new BasicInitContainerConfigurationStep(
    +      SPARK_JARS,
    +      SPARK_FILES,
    +      JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_PATH,
    +      podAndInitContainerBootstrap)
    +    val expectedDriverSparkConf = Map(
    +      JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH,
    +      INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar",
    +      INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt")
    +    val initContainerSpec = InitContainerSpec(
    +      Map.empty[String, String],
    +      Map.empty[String, String],
    +      new Container(),
    +      new Container(),
    +      new Pod,
    +      Seq.empty[HasMetadata])
    +    val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec)
    +    assert(expectedDriverSparkConf === returnContainerSpec.properties)
    +    assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME)
    +    assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME)
    --- End diff --
    
    `===`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @vanzin additionally to shortening names and removing unnecessary traits, we have further improved the code by removing unused/unnecessary class arguments and enhancing the Scaladocs on core classes. PTAL. Thanks for reviewing!


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911698
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Bootstraps an init-container for downloading remote dependencies. This is separated out from
    + * the init-container steps API because this component can be used to bootstrap init-containers
    + * for both the driver and executors.
    + */
    +private[spark] class InitContainerBootstrap(
    +    initContainerImage: String,
    +    imagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf) {
    +
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val customEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env =>
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > I think regardless of the abstractions we choose, we're eventually going to end up with many classes with unique constructors simply due to the complexity of the submission client itself.
    
    I disagree. For example, just to pick one example I spotted while glancing at the code quickly. There are many steps that take a `kubernetesResourceNamePrefix` as a constructor argument. Then the code that instantiates those steps needs to know each of those steps, and that they take that prefix as an argument to their constructor. I don't see that as abstracting much.
    
    Instead, for example, if you encapsulate the state of a "kubernetes submission" into some helper type that wraps SparkConf and other common things needed to submit an app, those steps could potentially just take that helper type as an argument to the abstract method that does the processing (e.g. `configureDriver`). There's less step-specific logic needed everywhere, and the list of arguments to constructors and methods shrinks a lot.
    
    So, for example, bad abstraction:
    
    ```
    val step1 = new Step1(/* lots of arguments */)
    val step2 = new Step2(/* lots of arguments, many repeated with step 1 */)
    val step3 = new Step3(/* lots of arguments, many repeated with step 1 */)
    val steps = Seq(step1, step2, step3)
    
    val something = steps.something { s => s.doStep(someObject) }
    val client = new ClientBuilder(/* yet another long list of things with repeated arguments */)
      .createClient(/* even more arguments */)
    ```
    
    Better abstraction:
    
    ```
    val steps = Seq(new Step1(), new Step2(), new Step3())
    
    val finalState = steps.something { s => s.doStep(someObjectThatTracksAllSubmissionState) }
    val client = new Client(finalState)
    ```
    
    You may not be able to clean up everything in the argument list, but you can get into a much better place. The current code is just full of repetition that could easily be avoided.
    
    > As with any abstraction, we can inline the orchestrator into the submission client, and then consider if that makes the Client do too many things, and also, what would the unit tests look like as a result? I like that we can test the orchestrator and its selection of which steps to apply independently
    
    That's the thing. That's not an abstraction, that's just code that lives in a separate method. You can test the "method that creates a list of steps" separately too without having to stash it in a completely separate type and create more coupling in your code. Abstraction is meant to reduce coupling, not increase it.
    
    An abstraction for an orchestrator would mean that there is a code path that can take a different orchestrator to achieve different things. For example, if you had code that started containers, and given a different orchestrator, could start either a driver or an executor, or even be used by tests to mock some behavior you want to verify. But that's not what you have. You just have code living in separate classes with the goal of making testing easier, but you can achieve the same thing without the code living in separate classes, and things would probably be simpler overall.
    
    The same argument can actually be made about steps. You could potentially have one big class that has each step as a separate method, and one method that calls all of the needed methods in the right order (that would be the "orchestrator" method). You could test each "step" individually because they wouldn't be calling each other. (Now I'm not against the "step" abstraction per se, I just think there's a lot of room for improvement in the current code.)


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157942449
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    I created https://issues.apache.org/jira/browse/SPARK-22839.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157320322
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,67 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] trait MountSecretsBootstrap {
    +
    +  /**
    +   * Mounts Kubernetes secrets as secret volumes into the given container in the given pod.
    +   *
    +   * @param pod the pod into which the secret volumes are being added.
    +   * @param container the container into which the secret volumes are being mounted.
    +   * @return the updated pod and container with the secrets mounted.
    +   */
    +  def mountSecrets(pod: Pod, container: Container): (Pod, Container)
    +}
    +
    +private[spark] class MountSecretsBootstrapImpl(
    +    secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap {
    +
    +  override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = {
    +    var podBuilder = new PodBuilder(pod)
    +    secretNamesToMountPaths.keys.foreach(name =>
    --- End diff --
    
    `.foreach { name =>`


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167537
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringReader
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +import org.apache.spark.util.Utils
    +
    +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite {
    +
    +  private val CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val CONFIG_MAP_KEY = "spark-init-config-map-key"
    +
    +  test("The init container bootstrap step should use all of the init container steps") {
    +    val baseDriverSpec = KubernetesDriverSpec(
    +      driverPod = new PodBuilder().build(),
    +      driverContainer = new ContainerBuilder().build(),
    +      driverSparkConf = new SparkConf(false),
    +      otherKubernetesResources = Seq.empty[HasMetadata])
    +    val initContainerSteps = Seq(
    +      FirstTestInitContainerConfigurationStep,
    +      SecondTestInitContainerConfigurationStep)
    +    val bootstrapStep = new DriverInitContainerBootstrapStep(
    +      initContainerSteps,
    +      CONFIG_MAP_NAME,
    +      CONFIG_MAP_KEY)
    +
    +    val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec)
    +
    +    assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala ===
    +      FirstTestInitContainerConfigurationStep.additionalLabels)
    +    val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
    +    assert(additionalDriverEnv.size === 1)
    +    assert(additionalDriverEnv.head.getName ===
    +      FirstTestInitContainerConfigurationStep.additionalMainContainerEnvKey)
    +    assert(additionalDriverEnv.head.getValue ===
    +      FirstTestInitContainerConfigurationStep.additionalMainContainerEnvValue)
    +
    +    assert(preparedDriverSpec.otherKubernetesResources.size === 2)
    +    assert(preparedDriverSpec.otherKubernetesResources.contains(
    +      FirstTestInitContainerConfigurationStep.additionalKubernetesResource))
    +    assert(preparedDriverSpec.otherKubernetesResources.exists {
    +      case configMap: ConfigMap =>
    +        val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME
    +        val configMapData = configMap.getData.asScala
    +        val hasCorrectNumberOfEntries = configMapData.size == 1
    +        val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY)
    +        val initContainerProperties = new Properties()
    +        Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) {
    +          initContainerProperties.load(_)
    +        }
    +        val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala
    +        val expectedInitContainerProperties = Map(
    +          SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyKey ->
    +            SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyValue)
    +        val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties
    +        hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties
    +      case _ => false
    --- End diff --
    
    Add empty line before this.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651909
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    +      .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " +
    +        "locations into the driver and executor pods.")
    +      .timeConf(TimeUnit.MINUTES)
    +      .createWithDefault(5)
    +
    +  val INIT_CONTAINER_MAX_THREAD_POOL_SIZE =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads")
    --- End diff --
    
    I think the current name is already pretty long. Adding `initContainer` makes it even longer without much added value. 


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158074477
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala ---
    @@ -0,0 +1,94 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringWriter
    +import java.util.Properties
    +
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata}
    +
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec}
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +
    +/**
    + * Configures the driver init-container that localizes remote dependencies into the driver pod.
    + * It applies the given InitContainerConfigurationSteps in the given order to produce a final
    + * InitContainerSpec that is then used to configure the driver pod with the init-container attached.
    + * It also builds a ConfigMap that will be mounted into the init-container. The ConfigMap carries
    + * configuration properties for the init-container.
    + */
    +private[spark] class DriverInitContainerBootstrapStep(
    +    steps: Seq[InitContainerConfigurationStep],
    --- End diff --
    
    I think we can address this as part of the refactoring work.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157893995
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    --- End diff --
    
    Could you remove "maybe" from these variables? There are some specific cases where "maybe" can help, but this is not one of them.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157911449
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala ---
    @@ -209,9 +213,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
             .build()
         }.getOrElse(executorContainer)
     
    -    new PodBuilder(executorPod)
    +    val (maybeSecretsMountedPod, maybeSecretsMountedContainer) =
    +      mountSecretsBootstrap.map { bootstrap =>
    +        bootstrap.mountSecrets(executorPod, containerWithLimitCores)
    --- End diff --
    
    Yes, true.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651953
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (initContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (initContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157443239
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringReader
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +import org.apache.spark.util.Utils
    +
    +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite {
    +
    +  private val CONFIG_MAP_NAME = "spark-init-config-map"
    +  private val CONFIG_MAP_KEY = "spark-init-config-map-key"
    +
    +  test("The init container bootstrap step should use all of the init container steps") {
    +    val baseDriverSpec = KubernetesDriverSpec(
    +      driverPod = new PodBuilder().build(),
    +      driverContainer = new ContainerBuilder().build(),
    +      driverSparkConf = new SparkConf(false),
    +      otherKubernetesResources = Seq.empty[HasMetadata])
    +    val initContainerSteps = Seq(
    +      FirstTestInitContainerConfigurationStep$,
    +      SecondTestInitContainerConfigurationStep$)
    +    val bootstrapStep = new DriverInitContainerBootstrapStep(
    +      initContainerSteps,
    +      CONFIG_MAP_NAME,
    +      CONFIG_MAP_KEY)
    +
    +    val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec)
    +
    +    assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala ===
    +      FirstTestInitContainerConfigurationStep$.additionalLabels)
    +    val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
    +    assert(additionalDriverEnv.size === 1)
    +    assert(additionalDriverEnv.head.getName ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey)
    +    assert(additionalDriverEnv.head.getValue ===
    +      FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue)
    +
    +    assert(preparedDriverSpec.otherKubernetesResources.size === 2)
    +    assert(preparedDriverSpec.otherKubernetesResources.contains(
    +      FirstTestInitContainerConfigurationStep$.additionalKubernetesResource))
    +    assert(preparedDriverSpec.otherKubernetesResources.exists {
    +      case configMap: ConfigMap =>
    +        val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME
    +        val configMapData = configMap.getData.asScala
    +        val hasCorrectNumberOfEntries = configMapData.size == 1
    +        val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY)
    +        val initContainerProperties = new Properties()
    +        Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) {
    +          initContainerProperties.load(_)
    +        }
    +        val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala
    +        val expectedInitContainerProperties = Map(
    +          SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey ->
    +            SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue)
    +        val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties
    +        hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties
    +      case _ => false
    +    })
    +
    +    val initContainers = preparedDriverSpec.driverPod.getSpec.getInitContainers
    +    assert(initContainers.size() === 1)
    +    val initContainerEnv = initContainers.get(0).getEnv.asScala
    +    assert(initContainerEnv.size === 1)
    +    assert(initContainerEnv.head.getName ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey)
    +    assert(initContainerEnv.head.getValue ===
    +      SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue)
    +
    +    val expectedSparkConf = Map(
    +      INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME,
    +      INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY,
    +      SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey ->
    +        SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue)
    +    assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf)
    +  }
    +}
    +
    +private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
    +
    +  val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue")
    +  val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY"
    +  val additionalMainContainerEnvValue = "TEST_ENV_MAIN_VALUE"
    +  val additionalKubernetesResource = new SecretBuilder()
    +    .withNewMetadata()
    +    .withName("test-secret")
    +    .endMetadata()
    +    .addToData("secret-key", "secret-value")
    +    .build()
    +
    +  override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
    +    val driverPod = new PodBuilder(initContainerSpec.driverPod)
    +      .editOrNewMetadata()
    +      .addToLabels(additionalLabels.asJava)
    +      .endMetadata()
    +      .build()
    +    val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer)
    +      .addNewEnv()
    +      .withName(additionalMainContainerEnvKey)
    +      .withValue(additionalMainContainerEnvValue)
    +      .endEnv()
    +      .build()
    +    initContainerSpec.copy(
    +      driverPod = driverPod,
    +      driverContainer = mainContainer,
    +      initContainerDependentResources = initContainerSpec.initContainerDependentResources ++
    +        Seq(additionalKubernetesResource))
    +  }
    +}
    +
    +private object SecondTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
    --- End diff --
    
    ditto.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158168278
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.UUID
    +
    +import com.google.common.base.Charsets
    +import com.google.common.io.Files
    +import org.mockito.Mockito
    +import org.scalatest.BeforeAndAfter
    +import org.scalatest.mockito.MockitoSugar._
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.util.Utils
    +
    +class SparkPodInitContainerSuite
    +  extends SparkFunSuite with BeforeAndAfter {
    +
    +  import SparkPodInitContainerSuite.createTempFile
    --- End diff --
    
    Is this only used in this class? If so you don't need the object, right? Just a private method would be enough.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > @liyinan926 we might be able to make the review smaller, though this makes some of the code we contribute on each push unusable on its own.
    
    @mccheah I think the biggest thing that makes reviewing this PR challenging is the way the code is structured, not the amount of changes. This PR is actually both shorter and conceptually simpler than the first two big ones, IMO. Unless we do a thorough refactoring of the code, I personally don't think splitting into smaller ones will help much on reducing the confusion. Splitting also risks making half baked feature into 2.3, which I personally don't think is a good idea.   


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327476
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -133,30 +132,78 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.docker.image")
    --- End diff --
    
    None of it is docker specific, can be `container` everywhere.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327297
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerBootstrapImpl, MountSecretsBootstrapImpl}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Returns the complete ordered list of steps required to configure the init-container. This is
    + * only used when there are remote application dependencies to localize.
    + */
    +private[spark] class InitContainerConfigurationStepsOrchestrator(
    --- End diff --
    
    Renamed to `InitContainerConfigOrchestrator` and similarly `DriverConfigurationStepsOrchestrator` to `DriverConfigOrchestrator`.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @mridulm We did have some discussions on resource localization at some point. This is a powerful mechanism when coupled with a resource staging server (in the future). There is a cost per-pod when using localization in this manner, but there is an alternative - using a docker image baked with dependencies accomplishes that, since k8s does node-level caching of the downloaded images, so, one would pay the cost only once per node.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167623
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala ---
    @@ -0,0 +1,47 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.apache.spark.deploy.k8s.MountSecretsBootstrap
    +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils}
    +
    +class DriverMountSecretsStepSuite extends SparkFunSuite {
    +
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/driver"
    +
    +  test("Mounts all given secrets") {
    +    val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false))
    +    val secretNamesToMountPaths = Map(
    +      SECRET_FOO -> SECRET_MOUNT_PATH,
    +      SECRET_BAR -> SECRET_MOUNT_PATH)
    +
    +    val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths)
    +    val mountSecretsStep = new DriverMountSecretsStep(mountSecretsBootstrap)
    +    val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec)
    +    val driverPodWithSecretsMounted = configuredDriverSpec.driverPod
    +    val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer
    +
    +    Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName =>
    --- End diff --
    
    `.foreach { volumeName =>`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85069/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158650027
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -132,30 +131,84 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.image")
    +      .doc("Image for the driver and executor's init-container for downloading dependencies.")
    +      .stringConf
    +      .createOptional
    +
    +  val INIT_CONTAINER_MOUNT_TIMEOUT =
    +    ConfigBuilder("spark.kubernetes.mountDependencies.timeout")
    --- End diff --
    
    nit: `spark.kubernetes.initContainer.mountDependencies.timeout`?


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85056 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85056/testReport)** for PR 19954 at commit [`340fa41`](https://github.com/apache/spark/commit/340fa4107cabdf325ec884855098e4ded3c666fa).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > I don't think they are independent as architecturally they make sense together and represent a single concern: enabling use of remote dependencies through init-containers. Missing any one of the three makes the feature unusable. I would also argue that it won't necessarily make review easier as reviewers need to mentally connect them together to make sense of each change set. If the general conclusion is that we should first refactor the code to achieve a better abstraction instead of getting this feature into 2.3, I can buy that. But I don't think we should tackle this as three components.
    
    It is true that introducing each of those components individually doesn't make the feature usable, but these parts are independent in the sense that they each have different classes and different sections of code. Furthermore the architectural considerations are different for each of them. There's some architectural connection between the submission client and the driver configuring the driver and the executor pods though given that we're trying to share code. But the init-container itself can be contributed as a completely separate push.
    
    Basically I'm wondering if we can reduce the diff in any way. Making the code itself simpler as we've been discussing is certainly the primary way.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327642
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    --- End diff --
    
    Sure, but that's not my point. If you have 10 jars and 10 files to download, the current code will only download 2 at a time. If you submit each jar / file separately, you'll download as many as your thread pool allows, and you can make that configurable.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157327539
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    --- End diff --
    
    This class actually handles more tasks in our fork. For example, is is also responsible for downloading from the resource staging server that hosts submission client dependencies. The resource staging server will be in a future PR.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157912757
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala ---
    @@ -116,10 +127,45 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    +    val mayBeInitContainerBootstrapStep =
    +      if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +        val orchestrator = new InitContainerConfigOrchestrator(
    +          sparkJars,
    +          sparkFiles,
    +          jarsDownloadPath,
    +          filesDownloadPath,
    +          imagePullPolicy,
    +          initContainerConfigMapName,
    +          INIT_CONTAINER_PROPERTIES_FILE_NAME,
    +          sparkConf)
    +        val bootstrapStep = new DriverInitContainerBootstrapStep(
    +          orchestrator.getAllConfigurationSteps,
    +          initContainerConfigMapName,
    +          INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +
    +        Some(bootstrapStep)
    +      } else {
    +        None
    +      }
    +
    +    val mayBeMountSecretsStep = if (secretNamesToMountPaths.nonEmpty) {
    --- 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 #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/84985/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167741
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala ---
    @@ -0,0 +1,95 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +import org.mockito.{Mock, MockitoAnnotations}
    +import org.mockito.Matchers.any
    +import org.mockito.Mockito.when
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +
    +class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val SPARK_JARS = Seq(
    +    "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
    +  private val SPARK_FILES = Seq(
    +    "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
    +  private val JARS_DOWNLOAD_PATH = "/var/data/jars"
    +  private val FILES_DOWNLOAD_PATH = "/var/data/files"
    +  private val POD_LABEL = Map("bootstrap" -> "true")
    +  private val INIT_CONTAINER_NAME = "init-container"
    +  private val DRIVER_CONTAINER_NAME = "driver-container"
    +
    +  @Mock
    +  private var podAndInitContainerBootstrap : InitContainerBootstrap = _
    +
    +  before {
    +    MockitoAnnotations.initMocks(this)
    +    when(podAndInitContainerBootstrap.bootstrapInitContainer(
    +      any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] {
    +      override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = {
    +        val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer])
    +        pod.copy(
    +          pod = new PodBuilder(pod.pod)
    +            .withNewMetadata()
    +            .addToLabels("bootstrap", "true")
    +            .endMetadata()
    +            .withNewSpec().endSpec()
    +            .build(),
    +          initContainer = new ContainerBuilder()
    +            .withName(INIT_CONTAINER_NAME)
    +            .build(),
    +          mainContainer = new ContainerBuilder()
    +            .withName(DRIVER_CONTAINER_NAME)
    +            .build()
    +        )}})
    +  }
    +
    +  test("Test of additionalDriverSparkConf with mix of remote files and jars") {
    --- End diff --
    
    remove "Test of"


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Jenkins, retest this please.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85013 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85013/testReport)** for PR 19954 at commit [`197882d`](https://github.com/apache/spark/commit/197882d787b602b80b0a9b500133463a95a9a691).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85028/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157351657
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    --- End diff --
    
    Updated to create one task per file/jar to download. Regarding the type of thread pool, we are using a `CachedThreadPool`, which I think makes sense as it can be expected that the tasks are not long-lived.  


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #84793 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/84793/testReport)** for PR 19954 at commit [`cd5e832`](https://github.com/apache/spark/commit/cd5e832a32c229d9f6b0376b01b6b92a0ff42bd3).
     * This patch **fails Scala style 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85126/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158168059
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala ---
    @@ -0,0 +1,57 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.MountSecretsBootstrap
    +import org.apache.spark.deploy.k8s.submit.SecretVolumeUtils
    +
    +class InitContainerMountSecretsStepSuite extends SparkFunSuite {
    +
    +  private val SECRET_FOO = "foo"
    +  private val SECRET_BAR = "bar"
    +  private val SECRET_MOUNT_PATH = "/etc/secrets/init-container"
    +
    +  test("Mounts all given secrets") {
    --- End diff --
    
    lower case


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158167791
  
    --- Diff: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala ---
    @@ -0,0 +1,95 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps.initcontainer
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +import org.mockito.{Mock, MockitoAnnotations}
    +import org.mockito.Matchers.any
    +import org.mockito.Mockito.when
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.BeforeAndAfter
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer}
    +import org.apache.spark.deploy.k8s.Config._
    +
    +class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter {
    +
    +  private val SPARK_JARS = Seq(
    +    "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
    +  private val SPARK_FILES = Seq(
    +    "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
    +  private val JARS_DOWNLOAD_PATH = "/var/data/jars"
    +  private val FILES_DOWNLOAD_PATH = "/var/data/files"
    +  private val POD_LABEL = Map("bootstrap" -> "true")
    +  private val INIT_CONTAINER_NAME = "init-container"
    +  private val DRIVER_CONTAINER_NAME = "driver-container"
    +
    +  @Mock
    +  private var podAndInitContainerBootstrap : InitContainerBootstrap = _
    +
    +  before {
    +    MockitoAnnotations.initMocks(this)
    +    when(podAndInitContainerBootstrap.bootstrapInitContainer(
    +      any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] {
    +      override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = {
    +        val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer])
    +        pod.copy(
    +          pod = new PodBuilder(pod.pod)
    +            .withNewMetadata()
    +            .addToLabels("bootstrap", "true")
    +            .endMetadata()
    +            .withNewSpec().endSpec()
    +            .build(),
    +          initContainer = new ContainerBuilder()
    +            .withName(INIT_CONTAINER_NAME)
    +            .build(),
    +          mainContainer = new ContainerBuilder()
    +            .withName(DRIVER_CONTAINER_NAME)
    +            .build()
    +        )}})
    +  }
    +
    +  test("Test of additionalDriverSparkConf with mix of remote files and jars") {
    +    val baseInitStep = new BasicInitContainerConfigurationStep(
    +      SPARK_JARS,
    +      SPARK_FILES,
    +      JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_PATH,
    +      podAndInitContainerBootstrap)
    +    val expectedDriverSparkConf = Map(
    +      JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH,
    +      FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH,
    +      INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar",
    +      INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt")
    +    val initContainerSpec = InitContainerSpec(
    +      Map.empty[String, String],
    +      Map.empty[String, String],
    +      new Container(),
    +      new Container(),
    +      new Pod,
    +      Seq.empty[HasMetadata])
    +    val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec)
    +    assert(expectedDriverSparkConf === returnContainerSpec.properties)
    +    assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME)
    --- End diff --
    
    `===`


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157322857
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala ---
    @@ -116,10 +122,53 @@ private[spark] class DriverConfigurationStepsOrchestrator(
           None
         }
     
    +    val mayBeInitContainerBootstrapStep =
    +      if (areAnyFilesNonContainerLocal(sparkJars ++ sparkFiles)) {
    +        val initContainerConfigurationStepsOrchestrator =
    +          new InitContainerConfigurationStepsOrchestrator(
    +            namespace,
    +            kubernetesResourceNamePrefix,
    +            sparkJars,
    +            sparkFiles,
    +            jarsDownloadPath,
    +            filesDownloadPath,
    +            dockerImagePullPolicy,
    +            allDriverLabels,
    +            initContainerConfigMapName,
    +            INIT_CONTAINER_PROPERTIES_FILE_NAME,
    +            submissionSparkConf)
    +        val initContainerConfigurationSteps =
    +          initContainerConfigurationStepsOrchestrator.getAllConfigurationSteps()
    +        val initContainerBootstrapStep =
    +          new DriverInitContainerBootstrapStep(
    +            initContainerConfigurationSteps,
    +            initContainerConfigMapName,
    +            INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +
    +        Some(initContainerBootstrapStep)
    +      } else {
    +        None
    +      }
    +
    +    val mayBeMountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) {
    +      val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths)
    +      Some(new DriverMountSecretsStep(mountSecretsBootstrap))
    +    } else {
    +      None
    +    }
    +
         Seq(
           initialSubmissionStep,
           driverAddressStep,
           kubernetesCredentialsStep) ++
    -      maybeDependencyResolutionStep.toSeq
    +      maybeDependencyResolutionStep.toSeq ++
    +      mayBeInitContainerBootstrapStep.toSeq ++
    +      mayBeMountSecretsStep.toSeq
    +  }
    +
    +  private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = {
    +    files.exists { uri =>
    +      Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local"
    --- End diff --
    
    `Utils.resolveURI(uri) != "local"`


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157145702
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          s"or is not a directory.")
    --- End diff --
    
    nit: remove `s`.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157899398
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala ---
    @@ -0,0 +1,94 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringWriter
    +import java.util.Properties
    +
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata}
    +
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec}
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +
    +/**
    + * Configures the driver init-container that localizes remote dependencies into the driver pod.
    + * It applies the given InitContainerConfigurationSteps in the given order to produce a final
    + * InitContainerSpec that is then used to configure the driver pod with the init-container attached.
    + * It also builds a ConfigMap that will be mounted into the init-container. The ConfigMap carries
    + * configuration properties for the init-container.
    + */
    +private[spark] class DriverInitContainerBootstrapStep(
    +    steps: Seq[InitContainerConfigurationStep],
    --- End diff --
    
    I don't think we want to be using a separate step hierarchy here. Originally we weren't sure how many different configurations the init-container could have. But now that we've had the init-container around for awhile we've only seen two variations: submitting local files vs. not submitting local files. I think we can inline these configurations into the step directly.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85122 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85122/testReport)** for PR 19954 at commit [`429146d`](https://github.com/apache/spark/commit/429146db5ce3986fceef4b6b3381c7bac2fbd97d).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157883353
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
    +
    +/**
    + * Bootstraps a driver or executor container or an init-container with needed secrets mounted.
    + */
    +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) {
    --- End diff --
    
    If I understand this class correctly, it seems like what you're trying to do here is to inject this logic into different steps that require this functionality. So the code that instantiates those steps needs to know about this dependency, and needs to know how to create both objects. Then the step implementation has to call this code.
    
    Instead, wouldn't it be cleaner to make the step inherit this functionality?
    
    e.g.
    
    ```
    trait MountSecretsBootstrap(args) {
      def mountSecrets(...) { }
    }
    
    class InitContainerMountSecretsStep extends InitContainerConfigurationStep with MountSecretsBootstrap {
    
    }
    ```
    
    The same comment could be made about the init container boostrap.
    
    But in both cases, I'm not sure this would work right now on the executor side, because as I mentioned it doesn't really use the same abstraction as the driver side. Which is kinda one of the problems with the current class hierarchy here...


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158651520
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (initContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    --- End diff --
    
    nit: `was not` -> `is not`


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157881479
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * Bootstraps an init-container for downloading remote dependencies. This is separated out from
    + * the init-container steps API because this component can be used to bootstrap init-containers
    + * for both the driver and executors.
    + */
    +private[spark] class InitContainerBootstrap(
    +    initContainerImage: String,
    +    imagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    configMapName: String,
    +    configMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf) {
    +
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val customEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { env =>
    +      new EnvVarBuilder()
    +        .withName(env._1)
    +        .withValue(env._2)
    +        .build()
    +    }
    +
    +    val initContainer = new ContainerBuilder(original.initContainer)
    +      .withName("spark-init")
    +      .withImage(initContainerImage)
    +      .withImagePullPolicy(imagePullPolicy)
    +      .addAllToEnv(customEnvVars.asJava)
    +      .addNewVolumeMount()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
    +        .endVolumeMount()
    +      .addToVolumeMounts(sharedVolumeMounts: _*)
    +      .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
    +      .build()
    +
    +    val podWithBasicVolumes = new PodBuilder(original.pod)
    +      .editSpec()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
    +        .withNewConfigMap()
    +          .withName(configMapName)
    +          .addNewItem()
    +            .withKey(configMapKey)
    +            .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME)
    +            .endItem()
    +          .endConfigMap()
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .addNewVolume()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withEmptyDir(new EmptyDirVolumeSource())
    +        .endVolume()
    +      .endSpec()
    +      .build()
    +
    +    val mainContainer = new ContainerBuilder(
    +      original.mainContainer)
    --- End diff --
    
    Fits in previous line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85094 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85094/testReport)** for PR 19954 at commit [`5b82fc0`](https://github.com/apache/spark/commit/5b82fc06a5fff2a3ff2062c46a21d069640eeec7).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157634511
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s.submit.steps
    +
    +import java.io.StringWriter
    +import java.util.Properties
    +
    +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata}
    +
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, KubernetesDriverSpec}
    +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
    +
    +/**
    + * Configures the init-container that bootstraps dependencies into the driver pod, including
    + * building a ConfigMap that will be mounted into the init-container. The ConfigMap carries
    + * configuration properties for the init-container.
    + */
    +private[spark] class DriverInitContainerBootstrapStep(
    +    initContainerConfigurationSteps: Seq[InitContainerConfigurationStep],
    --- End diff --
    
    Another place where variable names are unnecessarily long and harm readability. `initContainer` is already implicit given the name of the class. I have to parse the variable name every time and ignore the prefix to know which one it is.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85324 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85324/testReport)** for PR 19954 at commit [`785b90e`](https://github.com/apache/spark/commit/785b90e52580e9175896b22b00b23f30fbe020ef).


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158166916
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class SparkPodInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE)
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    --- End diff --
    
    Fits in previous line.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] add init-container bootstrappi...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    @mccheah @foxish I gave you push access to the fork I used for this PR. Feel free to push commits if you want. Please do let me know if you plan to address comments from @vanzin and make changes. Otherwise, I will address them later tonight. 


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    > @liyinan926 the code structure is the main issue, yes, but we can tackle the code structure more effectively by having a better decomposition of the review process as well. We have these three distinct components which are relatively independent. We can therefore separate out the three pieces and consider the architecture for each of them individually. 
    
    I don't think they are independent as architecturally they make sense together and represent a single concern: enabling use of remote dependencies through init-containers. Missing any one of the three makes the feature unusable. I would also argue that it won't necessarily make review easier as reviewers need to mentally connect them together to make sense of each change set. If the general conclusion is that we should *first* refactor the code to achieve a better abstraction instead of getting this feature into 2.3, I can buy that. But I don't think we should tackle this as three components. 
    
    @foxish @felixcheung @vanzin any thoughts?      


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85384/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157891341
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.{File, FileInputStream}
    +import java.util.Properties
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.collect.Maps
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
    +import org.apache.spark.util.Utils
    +
    +private[spark] object SparkConfPropertiesParser {
    +
    +  def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
    --- End diff --
    
    This could be in `ConfigurationUtils` instead of adding another object with a single method?
    
    I'd probably just rename that object to `KubernetesUtils` or something and also include the code from `InitContainerUtil` in there.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157246524
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    +  /**
    +   * Bootstraps an init-container that downloads dependencies to be used by a main container.
    +   */
    +  def bootstrapInitContainer(
    +      originalPodWithInitContainer: PodWithDetachedInitContainer)
    +  : PodWithDetachedInitContainer
    +}
    +
    +private[spark] class InitContainerBootstrapImpl(
    +    initContainerImage: String,
    +    dockerImagePullPolicy: String,
    +    jarsDownloadPath: String,
    +    filesDownloadPath: String,
    +    downloadTimeoutMinutes: Long,
    +    initContainerConfigMapName: String,
    +    initContainerConfigMapKey: String,
    +    sparkRole: String,
    +    sparkConf: SparkConf)
    +  extends InitContainerBootstrap {
    +
    +  override def bootstrapInitContainer(
    +      podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
    +    val sharedVolumeMounts = Seq[VolumeMount](
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
    +        .withMountPath(jarsDownloadPath)
    +        .build(),
    +      new VolumeMountBuilder()
    +        .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
    +        .withMountPath(filesDownloadPath)
    +        .build())
    +
    +    val initContainerCustomEnvVarKeyPrefix = sparkRole match {
    +      case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
    +      case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
    +      case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
    +    }
    +    val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
    +      .toSeq
    +      .map(env =>
    +        new EnvVarBuilder()
    +          .withName(env._1)
    +          .withValue(env._2)
    +          .build())
    +
    +    val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer)
    +      .withName(s"spark-init")
    --- 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/85219/
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157328425
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala ---
    @@ -0,0 +1,129 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.rest.k8s
    +
    +import java.io.File
    +import java.util.concurrent.TimeUnit
    +
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +
    +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf}
    +import org.apache.spark.deploy.SparkHadoopUtil
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +/**
    + * Process that fetches files from a resource staging server and/or arbitrary remote locations.
    + *
    + * The init-container can handle fetching files from any of those sources, but not all of the
    + * sources need to be specified. This allows for composing multiple instances of this container
    + * with different configurations for different download sources, or using the same container to
    + * download everything at once.
    + */
    +private[spark] class KubernetesSparkDependencyDownloadInitContainer(
    +    sparkConf: SparkConf,
    +    fileFetcher: FileFetcher) extends Logging {
    +
    +  private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
    +    ThreadUtils.newDaemonCachedThreadPool("download-executor"))
    +
    +  private val jarsDownloadDir = new File(
    +    sparkConf.get(JARS_DOWNLOAD_LOCATION))
    +  private val filesDownloadDir = new File(
    +    sparkConf.get(FILES_DOWNLOAD_LOCATION))
    +
    +  private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
    +  private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
    +
    +  private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
    +
    +  def run(): Unit = {
    +    val remoteJarsDownload = Future[Unit] {
    +      logInfo(s"Downloading remote jars: $remoteJars")
    +      downloadFiles(
    +        remoteJars,
    +        jarsDownloadDir,
    +        s"Remote jars download directory specified at $jarsDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    val remoteFilesDownload = Future[Unit] {
    +      logInfo(s"Downloading remote files: $remoteFiles")
    +      downloadFiles(
    +        remoteFiles,
    +        filesDownloadDir,
    +        s"Remote files download directory specified at $filesDownloadDir does not exist " +
    +          "or is not a directory.")
    +    }
    +    waitForFutures(
    --- End diff --
    
    Got it, will address this.


---

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


[GitHub] spark pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157248504
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala ---
    @@ -45,6 +45,60 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
           masterURL: String,
           scheduler: TaskScheduler): SchedulerBackend = {
         val sparkConf = sc.getConf
    +    val maybeInitContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME)
    +    val maybeInitContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF)
    +
    +    if (maybeInitContainerConfigMap.isEmpty) {
    +      logWarning("The executor's init-container config map was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    if (maybeInitContainerConfigMapKey.isEmpty) {
    +      logWarning("The executor's init-container config map key was not specified. Executors will " +
    +        "therefore not attempt to fetch remote or submitted dependencies.")
    +    }
    +
    +    // Only set up the bootstrap if they've provided both the config map key and the config map
    +    // name. The config map might not be provided if init-containers aren't being used to
    +    // bootstrap dependencies.
    +    val maybeInitContainerBootstrap = for {
    +      configMap <- maybeInitContainerConfigMap
    +      configMapKey <- maybeInitContainerConfigMapKey
    +    } yield {
    +      val initContainerImage = sparkConf
    +        .get(INIT_CONTAINER_DOCKER_IMAGE)
    +        .getOrElse(throw new SparkException(
    +          "Must specify the init-container Docker image when there are remote dependencies"))
    +      new InitContainerBootstrapImpl(
    +        initContainerImage,
    +        sparkConf.get(DOCKER_IMAGE_PULL_POLICY),
    +        sparkConf.get(JARS_DOWNLOAD_LOCATION),
    +        sparkConf.get(FILES_DOWNLOAD_LOCATION),
    +        sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT),
    +        configMap,
    +        configMapKey,
    +        SPARK_POD_EXECUTOR_ROLE,
    +        sparkConf)
    +    }
    +
    +    val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX)
    +    val mayBeMountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) {
    +      Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths))
    +    } else {
    +      None
    +    }
    +    // Mount user-specified executor secrets also into the executor's init-container. The
    +    // init-container may need credentials in the secrets to be able to download remote
    +    // dependencies. The executor's main container and its init-container share the secrets
    +    // because the init-container is sort of an implementation details and this sharing
    +    // avoids introducing a dedicated configuration property just for the init-container.
    +    val mayBeInitContainerMountSecretsBootstrap = if (maybeInitContainerBootstrap.nonEmpty &&
    --- End diff --
    
    Oh, I see. Thanks!


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85398 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85398/testReport)** for PR 19954 at commit [`28343fb`](https://github.com/apache/spark/commit/28343fb50310826bc9962e785f25d1af9b3c3f4a).


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r158164898
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala ---
    @@ -14,13 +14,49 @@
      * 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 java.io.File
     
    +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder}
    +
    +import org.apache.spark.SparkConf
     import org.apache.spark.util.Utils
     
    -private[spark] object KubernetesFileUtils {
    +private[spark] object KubernetesUtils {
    +
    +  /**
    +   * Extract and parse Spark configuration properties with a given name prefix and
    +   * return the result as a Map. Keys must not have more than one value.
    +   *
    +   * @param sparkConf Spark configuration
    +   * @param prefix the given property name prefix
    +   * @return a Map storing the configuration property keys and values
    +   */
    +  def parsePrefixedKeyValuePairs(
    +      sparkConf: SparkConf,
    +      prefix: String): Map[String, String] = {
    +    sparkConf.getAllWithPrefix(prefix).toMap
    +  }
    +
    +  def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = {
    +    opt1.foreach { _ => require(opt2.isEmpty, errMessage) }
    +  }
    +
    +  /**
    +   * Append the given init-container to a pod's list of init-containers..
    --- End diff --
    
    Too many periods.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    **[Test build #85028 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/85028/testReport)** for PR 19954 at commit [`e20e212`](https://github.com/apache/spark/commit/e20e21240f93e3308e552ad75fd789de7cb5dbf1).
     * 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 #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

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


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157641587
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala ---
    @@ -0,0 +1,128 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.deploy.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}
    +
    +import org.apache.spark.{SparkConf, SparkException}
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.Constants._
    +
    +/**
    + * This is separated out from the init-container steps API because this component can be reused to
    + * set up the init-container for executors as well.
    + */
    +private[spark] trait InitContainerBootstrap {
    --- End diff --
    
    > It's more idiomatic to mock a trait than a class 
    
    Why? You can mock classes just fine.


---

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


[GitHub] spark issue #19954: [SPARK-22757][Kubernetes] Enable use of remote dependenc...

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

    https://github.com/apache/spark/pull/19954
  
    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 pull request #19954: [SPARK-22757][Kubernetes] Enable use of remote de...

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

    https://github.com/apache/spark/pull/19954#discussion_r157318630
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -133,30 +132,78 @@ private[spark] object Config extends Logging {
     
       val JARS_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
    -      .doc("Location to download jars to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pod.")
    +      .doc("Location to download jars to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pod.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-jars")
     
       val FILES_DOWNLOAD_LOCATION =
         ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
    -      .doc("Location to download files to in the driver and executors. When using" +
    -        " spark-submit, this directory must be empty and will be mounted as an empty directory" +
    -        " volume on the driver and executor pods.")
    +      .doc("Location to download files to in the driver and executors. When using " +
    +        "spark-submit, this directory must be empty and will be mounted as an empty directory " +
    +        "volume on the driver and executor pods.")
           .stringConf
           .createWithDefault("/var/spark-data/spark-files")
     
    +  val INIT_CONTAINER_DOCKER_IMAGE =
    +    ConfigBuilder("spark.kubernetes.initContainer.docker.image")
    --- End diff --
    
    Mirroring the discussion in the other PR, are these really restricted to docker? Is it a required config?


---

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