You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ssuchter <gi...@git.apache.org> on 2018/02/28 20:16:19 UTC

[GitHub] spark pull request #20697: Initial checkin of k8s integration tests.

GitHub user ssuchter opened a pull request:

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

    Initial checkin of k8s integration tests.

    These tests were developed in the https://github.com/apache-spark-on-k8s/spark-integration repo
    by several contributors. This is a copy of the current state into the main apache spark repo.
    The only changes from the current spark-integration repo state are:
    * Move the files from the repo root into resource-managers/kubernetes/integration-tests
    * Add a reference to these tests in the root README.md
    * Fix a path reference in dev/dev-run-integration-tests.sh
    * Add a TODO in include/util.sh
    
    ## What changes were proposed in this pull request?
    
    Incorporation of Kubernetes integration tests.
    
    ## How was this patch tested?
    
    This code has its own unit tests, but the main purpose is to provide the integration tests.
    I tested this on my laptop by running dev/dev-run-integration-tests.sh --spark-tgz ~/spark-2.4.0-SNAPSHOT-bin--.tgz
    
    The spark-integration tests have already been running for months in AMPLab, here is an example:
    https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-scheduled-spark-integration-master/
    
    Please review http://spark.apache.org/contributing.html before opening a pull request.


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

    $ git pull https://github.com/ssuchter/spark ssuchter-k8s-integration-tests

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

    https://github.com/apache/spark/pull/20697.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 #20697
    
----
commit 2c5305128eb5d2e7847a0b6df67b7068d7f0ad8f
Author: Sean Suchter <ss...@...>
Date:   2018-02-28T20:10:46Z

    Initial checkin of k8s integration tests.
    These tests were developed in the https://github.com/apache-spark-on-k8s/spark-integration repo
    by several contributors. This is a copy of the current state into the main apache spark repo.
    The only changes from the current spark-integration repo state are:
    * Move the files from the repo root into resource-managers/kubernetes/integration-tests
    * Add a reference to these tests in the root README.md
    * Fix a path reference in dev/dev-run-integration-tests.sh
    * Add a TODO in include/util.sh

----


---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1157/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Fixed the bug. @mccheah I'd appreciate your eyes on commit 1d8a265, for both correctness and style. (I haven't used Scala before this project, so I'm very not confidence in the best way to do things.)


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @vanzin I fixed the title (thanks @liyinan926) with the bug.
    
    I'll work on integrating its build with the main build.
    
    In terms of integration with the dev/run-tests.py, we at most want to make it optional. These tests have an additional system requirement (minikube) that I wouldn't want to break existing systems that use run-tests.py. What do you think?
    
    With respect to the AMPLab builds - in previous discussing with @shaneknapp, the new Ubuntu-based systems are the only ones set up to run minikube, and jobs that exercise these tests are restricted to those systems. I think we should stick with that, because the direction (as I understand it) is to make all the systems follow that profile, and then we'll achieve the goal of having all systems able to run this integration test.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91172 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91172/testReport)** for PR 20697 at commit [`13721f6`](https://github.com/apache/spark/commit/13721f69a21c91c0b42a5471b50b8f53e4c7808f).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3600/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r192488928
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    I added in the non-Kerberos tests, but one of them (remote url fetching) is failing. I commented it out, but I intend to debug it before this submission and uncomment it.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172685889
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/CloudTestBackend.scala ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.integrationtest.backend.cloud
    +
    +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
    +
    +import org.apache.spark.deploy.k8s.integrationtest.Utils
    +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend
    +
    +private[spark] object CloudTestBackend extends IntegrationTestBackend {
    --- End diff --
    
    To make this PR smaller, can we start by only introducing the Minikube-backed tests, and then introduce the cloud ones in a separate push? Think that will make it easier to review.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1346/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172680200
  
    --- Diff: resource-managers/kubernetes/integration-tests/e2e/e2e-prow.sh ---
    @@ -0,0 +1,69 @@
    +#!/bin/bash
    +
    +# 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.
    +
    +### This script is used by Kubernetes Test Infrastructure to run integration tests.
    +### See documenation at https://github.com/kubernetes/test-infra/tree/master/prow
    +
    +set -ex
    +
    +# set cwd correctly
    +cd "$(dirname "$0")/../"
    +
    +# Include requisite scripts
    +source ./include/util.sh
    +
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +BRANCH="master"
    +SPARK_REPO="https://github.com/apache/spark"
    +SPARK_REPO_LOCAL_DIR="$TEST_ROOT_DIR/target/spark"
    +
    +## Install basic dependencies
    +## These are for the kubekins-e2e environment in https://github.com/kubernetes/test-infra/tree/master/images/kubekins-e2e
    +echo "deb http://http.debian.net/debian jessie-backports main" >> /etc/apt/sources.list
    +apt-get update && apt-get install -y curl wget git tar uuid-runtime
    --- End diff --
    
    Usage of this script should specifically be documented. From what I understand, this is for cloud-based testing? The automatic installation could be accidentally called.


---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1157/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r193957378
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    I think we should submit without that test, and then fix it later. Ilan is waiting for this to merge.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194128687
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.Closeable
    +import java.net.URI
    +
    +import org.apache.spark.internal.Logging
    +
    +object Utils extends Logging {
    +
    +  def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = {
    +    val resource = createResource
    +    try f.apply(resource) finally resource.close()
    +  }
    +
    +  def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = {
    --- End diff --
    
    Just did a quick search for this and I don't think it's used.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Ok, merging to master. @ifilonenko please add the Pyspark tests on top of this work also in a separate patch.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91400 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91400/testReport)** for PR 20697 at commit [`b936953`](https://github.com/apache/spark/commit/b936953c871226ae8a2ccc7caa6096e9fc38c317).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    You can do that in `run-tests.py`, I believe. At the very least, you can check the title.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #88798 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88798/testReport)** for PR 20697 at commit [`84a7779`](https://github.com/apache/spark/commit/84a7779f50a0e4d4ca3ef01f5c3e430c681e569c).


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r192447027
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    Hm, I don't know what happened to the others, I didn't mean to delete them. Looking...


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1360/
    Test FAILed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I saw that @mccheah put a comment in the e2e-prow.sh that we don't need it here, so I took that as an answer to my previous question.
    
    I think I got rid of the cloning and building Spark source code. I also updated README.md - PTAL.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3672/
    Test FAILed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    > If not all hosts currently can run them, then it's ok to postpone this.
    
    That's exactly what I'm proposing too, so it seems like we agree. I also agree, btw, on the future state - when all hosts can run this, then incorporating into dev/run-tests.py is a good idea.
    
    I'm going to proceed by fixing all other open issues, and ping when this can get another review, then.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3674/
    Test FAILed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1351/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91236 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91236/testReport)** for PR 20697 at commit [`1d8a265`](https://github.com/apache/spark/commit/1d8a265d13b65dcec8db11a5be09d4a029037d2c).
     * 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 pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172691439
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/CloudTestBackend.scala ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.integrationtest.backend.cloud
    +
    +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
    +
    +import org.apache.spark.deploy.k8s.integrationtest.Utils
    +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend
    +
    +private[spark] object CloudTestBackend extends IntegrationTestBackend {
    --- End diff --
    
    Certainly seems possible; I agree it would aid review. There's a bunch of changes I need to make, I'll include this point.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172707045
  
    --- Diff: resource-managers/kubernetes/integration-tests/include/util.sh ---
    @@ -0,0 +1,43 @@
    +#!/usr/bin/env bash
    +
    +# 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.
    +
    +clone_build_spark() {
    --- End diff --
    
    Agree, this code is something that should be removed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1867/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172679601
  
    --- Diff: resource-managers/kubernetes/integration-tests/include/util.sh ---
    @@ -0,0 +1,43 @@
    +#!/usr/bin/env bash
    +
    +# 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.
    +
    +clone_build_spark() {
    --- End diff --
    
    We shouldn't ever be cloning Spark anymore, now that the tests would be run from an existing repository, yeah?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3714/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91310 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91310/testReport)** for PR 20697 at commit [`14f2a19`](https://github.com/apache/spark/commit/14f2a19cd6d572eb7cd22f80396a8d39d9420166).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Any more issues that should be addressed now? Is this ready for merge?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3764/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I'll work on Matt's comments from Friday next.
    
    Here's the output (after the bugfix) from running against mainline:
    
    ```
    MBP:~/src/ssuchter-spark% git remote get-url origin
    https://github.com/ssuchter/spark.git
    
    MBP:~/src/ssuchter-spark% git remote get-url upstream
    git://github.com/apache/spark.git
    
    1d8a265d13 (HEAD -> ssuchter-k8s-integration-tests, origin/ssuchter-k8s-integration-tests) Fix a bug in KubernetesTestComponents - don't an an empty string for zero arguments
    65347b319a Merge branch 'ssuchter-k8s-integration-tests' of https://github.com/ssuchter/spark into ssuchter-k8s-integration-tests
    1a531abcf6 Remove unused code relating to Kerberos, which doesn't belong in this PR
    3ba6ffb5f2 Remove e2e-prow.sh, which isn't appropriate for this PR
    9e64f43b62 Remove unnecessary cloning and building code for the Spark repo
    e6bd56325d Update README.md excluding cloning and building logic
    e70f3bea3d Remove K8s cloud-based backend testing support from this PR
    a0023b2f33 Remove config options that were only used during repo clone process
    e55b8a723e Remove repository cloning behavior and allow script to be called from other directories
    9b0eede244 Fixes for scala style
    f29679ef56 Ignore dist/ for style checks
    3615953bea Fix scala style issues
    bef586f740 Remove LICENSE and copy of mvn wrapper script. Rewrite path for calling mvn wrapper script.
    81c7a66ad6 Make k8s integration tests build when top-level kubernetes profile selected
    365d6bc65d Initial checkin of k8s integration tests. These tests were developed in the https://github.com/apache-spark-on-k8s/spark-integration repo by several contributors. This is a copy of the current state into the main apache spark repo. The only changes from the current spark-integration repo state are: * Move the files from the repo root into resource-managers/kubernetes/integration-tests * Add a reference to these tests in the root README.md * Fix a path reference in dev/dev-run-integration-tests.sh * Add a TODO in include/util.sh
    dbce275784 Remove unused code relating to Kerberos, which doesn't belong in this PR
    5ffa464c65 Remove e2e-prow.sh, which isn't appropriate for this PR
    13721f69a2 Remove unnecessary cloning and building code for the Spark repo
    ba720733fa Update README.md excluding cloning and building logic
    1b1528a504 (upstream/master, origin/master, origin/HEAD, master) [SPARK-24366][SQL] Improving of error messages for type converting
    
    MBP:~/src/ssuchter-spark% echo $REVISION
    1d8a265d13
    MBP:~/src/ssuchter-spark% echo $DATE
    20180528
    
    MBP:~/src/ssuchter-spark% ./dev/make-distribution.sh --name ${DATE}-${REVISION} --tgz -DzincPort=${ZINC_PORT} -Phadoop-2.7 -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver
    +++ dirname ./dev/make-distribution.sh
    ++ cd ./dev/..
    ++ pwd
    + SPARK_HOME=/Users/ssuchter/src/ssuchter-spark
    + DISTDIR=/Users/ssuchter/src/ssuchter-spark/dist
    + MAKE_TGZ=false
    + MAKE_PIP=false
    + MAKE_R=false
    …
    + TARDIR_NAME=spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    + TARDIR=/Users/ssuchter/src/ssuchter-spark/spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    + rm -rf /Users/ssuchter/src/ssuchter-spark/spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    + cp -r /Users/ssuchter/src/ssuchter-spark/dist /Users/ssuchter/src/ssuchter-spark/spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    + tar czf spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13.tgz -C /Users/ssuchter/src/ssuchter-spark spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    + rm -rf /Users/ssuchter/src/ssuchter-spark/spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13
    
    MBP:~/src/ssuchter-spark% ./resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh --spark-tgz ~/src/ssuchter-spark/spark-2.4.0-SNAPSHOT-bin-20180528-1d8a265d13.tgz
    Using `mvn` from path: /usr/local/bin/mvn
    [INFO] Scanning for projects...
    [INFO]
    [INFO] ------------------------------------------------------------------------
    [INFO] Building Spark Project Kubernetes Integration Tests 2.4.0-SNAPSHOT
    [INFO] ------------------------------------------------------------------------
    [INFO]
    [INFO] --- maven-enforcer-plugin:3.0.0-M1:enforce (enforce-versions) @ spark-kubernetes-integration-tests_2.11 ---
    …
    Successfully tagged kubespark/spark:68388D3B-6FAC-4E59-8AED-8604AA437C2D
    /Users/ssuchter/src/ssuchter-spark/resource-managers/kubernetes/integration-tests
    [INFO]
    [INFO] --- scalatest-maven-plugin:1.0:test (integration-test) @ spark-kubernetes-integration-tests_2.11 ---
    Discovery starting.
    Discovery completed in 118 milliseconds.
    Run starting. Expected test count is: 1
    KubernetesSuite:
    - Run SparkPi with no resources
    Run completed in 54 seconds, 202 milliseconds.
    Total number of tests run: 1
    Suites: completed 2, aborted 0
    Tests: succeeded 1, failed 0, canceled 0, ignored 0, pending 0
    All tests passed.
    [INFO] ------------------------------------------------------------------------
    [INFO] BUILD SUCCESS
    [INFO] ------------------------------------------------------------------------
    [INFO] Total time: 02:40 min
    [INFO] Finished at: 2018-05-28T20:21:28-07:00
    [INFO] Final Memory: 37M/470M
    [INFO] ------------------------------------------------------------------------
    
    ```
    
    



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Hm, that wasn't the answer I expected to hear. It leads me to another question:
    
    If we don't have developers using this script directly, what's the value of trying to incorporate the running of the Kubernetes integration tests into this program? Because of the AMPLab setup, we're going to have to have special Jenkins jobs dedicated to the running of the K8s integration tests. This is because not all hosts can run these integration tests, so therefore we have special jobs that are tied to the specific hosts that have the right setup to run the minikube VMs. So it's just going to be those jobs that call dev/run-tests.py with the special non-default argument to run these tests. Those jenkins jobs could just as easily run a different script.
    
    An alternative way to take my argument is that we should delay the integration into run-tests.py until after **all** of the AMPLab hosts can run these tests. Since that's not today, it wouldn't need to be in this initial PR, and could have its own PR.
    
    Thoughts? Thanks, Sean


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3602/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194124004
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a very long application name.") {
    +    sparkAppConf.set("spark.app.name", "long" * 40)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a master URL without a scheme.") {
    +    val url = kubernetesTestComponents.kubernetesClient.getMasterUrl
    +    val k8sMasterUrl = if (url.getPort < 0) {
    +      s"k8s://${url.getHost}"
    +    } else {
    +      s"k8s://${url.getHost}:${url.getPort}"
    +    }
    +    sparkAppConf.set("spark.master", k8sMasterUrl)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with an argument.") {
    +    runSparkPiAndVerifyCompletion(appArgs = Array("5"))
    +  }
    +
    +  test("Run SparkPi with custom labels, annotations, and environment variables.") {
    +    sparkAppConf
    +      .set("spark.kubernetes.driver.label.label1", "label1-value")
    +      .set("spark.kubernetes.driver.label.label2", "label2-value")
    +      .set("spark.kubernetes.driver.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.driver.annotation.annotation2", "annotation2-value")
    +      .set("spark.kubernetes.driverEnv.ENV1", "VALUE1")
    +      .set("spark.kubernetes.driverEnv.ENV2", "VALUE2")
    +      .set("spark.kubernetes.executor.label.label1", "label1-value")
    +      .set("spark.kubernetes.executor.label.label2", "label2-value")
    +      .set("spark.kubernetes.executor.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.executor.annotation.annotation2", "annotation2-value")
    +      .set("spark.executorEnv.ENV1", "VALUE1")
    +      .set("spark.executorEnv.ENV2", "VALUE2")
    +
    +    runSparkPiAndVerifyCompletion(
    +      driverPodChecker = (driverPod: Pod) => {
    +        doBasicDriverPodCheck(driverPod)
    +        checkCustomSettings(driverPod)
    +      },
    +      executorPodChecker = (executorPod: Pod) => {
    +        doBasicExecutorPodCheck(executorPod)
    +        checkCustomSettings(executorPod)
    +      })
    +  }
    +
    +  // TODO(ssuchter): Enable the below after debugging
    --- End diff --
    
    Why is this broken? It should have been working with the init-container PR that was merged?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3714/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1363/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1346/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @mccheah If you want to merge, and then I can fix the commented out test in another PR, that's ok too.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91310 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91310/testReport)** for PR 20697 at commit [`14f2a19`](https://github.com/apache/spark/commit/14f2a19cd6d572eb7cd22f80396a8d39d9420166).


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191966223
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    Ok, it's important for me to be clear here. There are currently two PRBs. This will continue in the immediate future.
    
    1. General Spark PRB, mainly for unit tests. This can run on all hosts.
    
    2. K8s integration-specific PRB. This early-outs on many PRs that don't seem relevant. This is specifically for running K8s integration tests, and can only run on some hosts.
    
    Because of the host restriction issue, these are two separate PRBs.
    
    It is definitely true that each one of these will build the main Spark jars separately, so that 11 minute time will be spent twice. Since the K8s-integration PRB is only doing this on a small set of PRs, it's not a significant cost to the Jenkins infrastructure.
    
    Within the K8s-integration PRB, the entire maven reactor is only built once, during the make distribution step. The integration test step doesn't rebuild it.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #88798 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88798/testReport)** for PR 20697 at commit [`84a7779`](https://github.com/apache/spark/commit/84a7779f50a0e4d4ca3ef01f5c3e430c681e569c).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91351 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91351/testReport)** for PR 20697 at commit [`67cb6e2`](https://github.com/apache/spark/commit/67cb6e21cd4726ae398c5f0045b1d758fb6a8238).


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @mccheah will do. also will investigate the remote test issue. 


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191965147
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    Sorry, I mean if the Spark PRB will try to build the entire maven reactor twice - once for unit tests and once for integration tests. The TGZ bundling in and of itself I agree should be fast if the jars are already built by the maven reactor. But it's unclear to me if we'll end up building jars redundantly here.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Barring further feedback, resolving a two issues should have this ready to merge:
    1. move cloud-backed testing to a new PR
    1. remove the repository clone logic



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    FYI - I just rebased to an updated master.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3575/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3631/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r192274844
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    Why not move all our tests over @ssuchter ? 


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191568423
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    Can we just use the repository and not require a tarball?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91260 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91260/testReport)** for PR 20697 at commit [`cfb8ee9`](https://github.com/apache/spark/commit/cfb8ee94e11b4871f9b8c7db4774bdb6cb42c40e).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91406 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91406/testReport)** for PR 20697 at commit [`4c5677a`](https://github.com/apache/spark/commit/4c5677a61fd940b818d81469e6640cb45f00ce58).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I agree with your point about using a non-tgz version, but I'd prefer to do that as a separate change. (However, I reserve my right to change this stance if it turns out to be easier to make the other changes to this PR in a way that runs without the .tgz file)


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3522/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3471/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191033974
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    +    <commons-logging.version>1.1.1</commons-logging.version>
    +    <docker-client.version>5.0.2</docker-client.version>
    +    <download-maven-plugin.version>1.3.0</download-maven-plugin.version>
    +    <exec-maven-plugin.version>1.4.0</exec-maven-plugin.version>
    +    <extraScalaTestArgs></extraScalaTestArgs>
    +    <guava.version>18.0</guava.version>
    +    <jsr305.version>1.3.9</jsr305.version>
    +    <kubernetes-client.version>3.0.0</kubernetes-client.version>
    +    <log4j.version>1.2.17</log4j.version>
    +    <scala.version>2.11.8</scala.version>
    +    <scala.binary.version>2.11</scala.binary.version>
    +    <scala-maven-plugin.version>3.2.2</scala-maven-plugin.version>
    +    <scalatest.version>2.2.6</scalatest.version>
    +    <scalatest-maven-plugin.version>1.0</scalatest-maven-plugin.version>
    +    <slf4j-log4j12.version>1.7.24</slf4j-log4j12.version>
    +    <sbt.project.name>kubernetes-integration-tests</sbt.project.name>
    +    <spark.kubernetes.test.unpackSparkDir>${project.build.directory}/spark-dist-unpacked</spark.kubernetes.test.unpackSparkDir>
    +    <spark.kubernetes.test.imageTag>N/A</spark.kubernetes.test.imageTag>
    +    <spark.kubernetes.test.imageTagFile>${project.build.directory}/imageTag.txt</spark.kubernetes.test.imageTagFile>
    +    <spark.kubernetes.test.deployMode>minikube</spark.kubernetes.test.deployMode>
    +    <spark.kubernetes.test.imageRepo>docker.io/kubespark</spark.kubernetes.test.imageRepo>
    +    <test.exclude.tags></test.exclude.tags>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes Integration Tests</name>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>commons-logging</groupId>
    +      <artifactId>commons-logging</artifactId>
    +      <version>${commons-logging.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.code.findbugs</groupId>
    +      <artifactId>jsr305</artifactId>
    +      <version>${jsr305.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <scope>test</scope>
    +      <!-- For compatibility with Docker client. Should be fine since this is just for tests.-->
    +      <version>${guava.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.spotify</groupId>
    +      <artifactId>docker-client</artifactId>
    +      <version>${docker-client.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    +      <version>${kubernetes-client.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>log4j</groupId>
    +      <artifactId>log4j</artifactId>
    +      <version>${log4j.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.apache.commons</groupId>
    +      <artifactId>commons-lang3</artifactId>
    +      <version>${commons-lang3.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.scala-lang</groupId>
    +      <artifactId>scala-library</artifactId>
    +      <version>${scala.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.scalatest</groupId>
    +      <artifactId>scalatest_${scala.binary.version}</artifactId>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.slf4j</groupId>
    +      <artifactId>slf4j-log4j12</artifactId>
    +      <version>${slf4j-log4j12.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +  </dependencies>
    +
    +  <build>
    +    <plugins>
    +      <plugin>
    +        <groupId>net.alchim31.maven</groupId>
    --- End diff --
    
    More things which might be provided by the parent pom. In general, we should audit this pom.xml and make sure we're not being redundant.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91402 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91402/testReport)** for PR 20697 at commit [`845cba1`](https://github.com/apache/spark/commit/845cba1db95293d7962fb6029c46e006a5da46a0).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91402 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91402/testReport)** for PR 20697 at commit [`845cba1`](https://github.com/apache/spark/commit/845cba1db95293d7962fb6029c46e006a5da46a0).


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    In much of the old integration test setup, we were cloning and building Spark into a TGZ. I wonder if we can skip both of these steps. We shouldn't ever have to clone Spark, as we have the repository here. Building Spark is a bit trickier, but we might be able to get around that too.
    
    In the `bin/` directory, all of the scripts can run against a Spark repository clone, without the repository having been bundled into a tgz. If we look at `bin/spark-class`, we notice the classpath can be picked up from the `jars` directory from the build output of the `assembly` project. I wonder if we can do something similar here - instead of relying on the built tgz itself, intelligently build all the dependents of the integration test into the assembly, and pick up the raw Dockerfiles from the repository layout.
    
    Can we look into a non-tgz dependent version of what we have here?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #88739 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88739/testReport)** for PR 20697 at commit [`84a7779`](https://github.com/apache/spark/commit/84a7779f50a0e4d4ca3ef01f5c3e430c681e569c).


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3473/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3601/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3631/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1891/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r192466840
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    There are two classes of additional tests, Kerberos related ones and non-Kerberos related ones. Per your comment on 2/28, I am leaving the Kerberos related ones out of this PR (I guess we'll do them in another PR?) but re-adding the non-Kerberos ones back in now.


---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194150624
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.Closeable
    +import java.net.URI
    +
    +import org.apache.spark.internal.Logging
    +
    +object Utils extends Logging {
    +
    +  def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = {
    +    val resource = createResource
    +    try f.apply(resource) finally resource.close()
    +  }
    +
    +  def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = {
    +    var originalThrowable: Throwable = null
    +    try {
    +      block
    +    } catch {
    +      case t: Throwable =>
    +        // Purposefully not using NonFatal, because even fatal exceptions
    +        // we don't want to have our finallyBlock suppress
    +        originalThrowable = t
    +        throw originalThrowable
    +    } finally {
    +      try {
    +        finallyBlock
    +      } catch {
    +        case t: Throwable =>
    +          if (originalThrowable != null) {
    +            originalThrowable.addSuppressed(t)
    +            logWarning(s"Suppressing exception in finally: " + t.getMessage, t)
    +            throw originalThrowable
    +          } else {
    +            throw t
    +          }
    +      }
    +    }
    +  }
    +
    +  def checkAndGetK8sMasterUrl(rawMasterURL: String): String = {
    --- End diff --
    
    I agree. I removed it. I did a series of greps and removed things that were't used.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @vanzin I'm working on the run-tests.py integration. It seems to be failing on a bunch of comments on the file dist/python/docs/conf.py
    
    ./dist/python/docs/conf.py:258:1: E265 block comment should start with '# '
    ./dist/python/docs/conf.py:261:1: E265 block comment should start with '# '
    ./dist/python/docs/conf.py:264:1: E265 block comment should start with '# '
    
    This change isn't involved with that file. Should I be fixing that file, or is there a way that people typically run run-tests.py to skip it? (If there's a place I should be reading about this, please feel free to point me there.)


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3628/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I won't really have time to look more closely at this until next week, at least, but I did notice an old comment from @ifilonenko that you did not address.
    
    If you could also post the output of running the tests against the current master in a comment, that would be good.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191031379
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,391 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a very long application name.") {
    +    sparkAppConf.set("spark.app.name", "long" * 40)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a master URL without a scheme.") {
    +    val url = kubernetesTestComponents.kubernetesClient.getMasterUrl
    +    val k8sMasterUrl = if (url.getPort < 0) {
    +      s"k8s://${url.getHost}"
    +    } else {
    +      s"k8s://${url.getHost}:${url.getPort}"
    +    }
    +    sparkAppConf.set("spark.master", k8sMasterUrl)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with an argument.") {
    +    runSparkPiAndVerifyCompletion(appArgs = Array("5"))
    +  }
    +
    +  test("Run SparkPi with custom labels, annotations, and environment variables.") {
    +    sparkAppConf
    +      .set("spark.kubernetes.driver.label.label1", "label1-value")
    +      .set("spark.kubernetes.driver.label.label2", "label2-value")
    +      .set("spark.kubernetes.driver.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.driver.annotation.annotation2", "annotation2-value")
    +      .set("spark.kubernetes.driverEnv.ENV1", "VALUE1")
    +      .set("spark.kubernetes.driverEnv.ENV2", "VALUE2")
    +      .set("spark.kubernetes.executor.label.label1", "label1-value")
    +      .set("spark.kubernetes.executor.label.label2", "label2-value")
    +      .set("spark.kubernetes.executor.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.executor.annotation.annotation2", "annotation2-value")
    +      .set("spark.executorEnv.ENV1", "VALUE1")
    +      .set("spark.executorEnv.ENV2", "VALUE2")
    +
    +    runSparkPiAndVerifyCompletion(
    +      driverPodChecker = (driverPod: Pod) => {
    +        doBasicDriverPodCheck(driverPod)
    +        checkCustomSettings(driverPod)
    +      },
    +      executorPodChecker = (executorPod: Pod) => {
    +        doBasicExecutorPodCheck(executorPod)
    +        checkCustomSettings(executorPod)
    +      })
    +  }
    +
    +  test("Run SparkPi with a test secret mounted into the driver and executor pods") {
    +    val secretName = TEST_SECRET_NAME_PREFIX + UUID.randomUUID().toString.replaceAll("-", "")
    +    createTestSecret(secretName)
    +
    +    sparkAppConf
    +      .set(s"spark.kubernetes.driver.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +      .set(s"spark.kubernetes.executor.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +
    +    try {
    +      runSparkPiAndVerifyCompletion(
    +        driverPodChecker = (driverPod: Pod) => {
    +          doBasicDriverPodCheck(driverPod)
    +          checkTestSecret(secretName, driverPod)
    +        },
    +        executorPodChecker = (executorPod: Pod) => {
    +          doBasicExecutorPodCheck(executorPod)
    +          checkTestSecret(secretName, executorPod)
    +        })
    +    } finally {
    +      deleteTestSecret(secretName)
    +    }
    +  }
    +
    +  test("Run PageRank using remote data file") {
    +    sparkAppConf
    +      .set("spark.kubernetes.mountDependencies.filesDownloadDir",
    +        CONTAINER_LOCAL_FILE_DOWNLOAD_PATH)
    +      .set("spark.files", REMOTE_PAGE_RANK_DATA_FILE)
    +    runSparkPageRankAndVerifyCompletion(
    +      appArgs = Array(CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE))
    +  }
    +
    +  test("Run PageRank using remote data file with test secret mounted into the driver and " +
    +    "executors") {
    +    val secretName = TEST_SECRET_NAME_PREFIX + UUID.randomUUID().toString.replaceAll("-", "")
    +    createTestSecret(secretName)
    +
    +    sparkAppConf
    +      .set("spark.kubernetes.mountDependencies.filesDownloadDir",
    +        CONTAINER_LOCAL_FILE_DOWNLOAD_PATH)
    +      .set("spark.files", REMOTE_PAGE_RANK_DATA_FILE)
    +      .set(s"spark.kubernetes.driver.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +      .set(s"spark.kubernetes.executor.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +
    +    try {
    +      runSparkPageRankAndVerifyCompletion(
    +        appArgs = Array(CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE),
    +        driverPodChecker = (driverPod: Pod) => {
    +          doBasicDriverPodCheck(driverPod)
    +          checkTestSecret(secretName, driverPod, withInitContainer = true)
    +        },
    +        executorPodChecker = (executorPod: Pod) => {
    +          doBasicExecutorPodCheck(executorPod)
    +          checkTestSecret(secretName, executorPod, withInitContainer = true)
    +        })
    +    } finally {
    +      deleteTestSecret(secretName)
    +    }
    +  }
    +
    +  private def runSparkPiAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String] = Array.empty[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PI_MAIN_CLASS,
    +      Seq("Pi is roughly 3"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkPageRankAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PAGE_RANK_MAIN_CLASS,
    +      Seq("1 has rank", "2 has rank", "3 has rank", "4 has rank"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkApplicationAndVerifyCompletion(
    +      appResource: String,
    +      mainClass: String,
    +      expectedLogOnCompletion: Seq[String],
    +      appArgs: Array[String],
    +      driverPodChecker: Pod => Unit,
    +      executorPodChecker: Pod => Unit,
    +      appLocator: String): Unit = {
    +    val appArguments = SparkAppArguments(
    +      mainAppResource = appResource,
    +      mainClass = mainClass,
    +      appArgs = appArgs)
    +    SparkAppLauncher.launch(appArguments, sparkAppConf, TIMEOUT.value.toSeconds.toInt, sparkHomeDir)
    +
    +    val driverPod = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "driver")
    +      .list()
    +      .getItems
    +      .get(0)
    +    driverPodChecker(driverPod)
    +
    +    val executorPods = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "executor")
    +      .list()
    +      .getItems
    +    executorPods.asScala.foreach { pod =>
    +      executorPodChecker(pod)
    +    }
    +
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      expectedLogOnCompletion.foreach { e =>
    +        assert(kubernetesTestComponents.kubernetesClient
    +          .pods()
    +          .withName(driverPod.getMetadata.getName)
    +          .getLog
    +          .contains(e), "The application did not complete.")
    +      }
    +    }
    +  }
    +
    +  private def doBasicDriverPodCheck(driverPod: Pod): Unit = {
    +    assert(driverPod.getMetadata.getName === driverPodName)
    +    assert(driverPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver")
    +  }
    +
    +  private def doBasicExecutorPodCheck(executorPod: Pod): Unit = {
    +    assert(executorPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(executorPod.getSpec.getContainers.get(0).getName === "executor")
    +  }
    +
    +  private def checkCustomSettings(pod: Pod): Unit = {
    +    assert(pod.getMetadata.getLabels.get("label1") === "label1-value")
    +    assert(pod.getMetadata.getLabels.get("label2") === "label2-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation1") === "annotation1-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation2") === "annotation2-value")
    +
    +    val container = pod.getSpec.getContainers.get(0)
    +    val envVars = container
    +      .getEnv
    +      .asScala
    +      .map { env =>
    +        (env.getName, env.getValue)
    +      }
    +      .toMap
    +    assert(envVars("ENV1") === "VALUE1")
    +    assert(envVars("ENV2") === "VALUE2")
    +  }
    +
    +  private def deleteDriverPod(): Unit = {
    +    kubernetesTestComponents.kubernetesClient.pods().withName(driverPodName).delete()
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      assert(kubernetesTestComponents.kubernetesClient
    +        .pods()
    +        .withName(driverPodName)
    +        .get() == null)
    +    }
    +  }
    +
    +  private def createTestSecret(secretName: String): Unit = {
    +    kubernetesTestComponents.kubernetesClient.secrets
    +      .createNew()
    +      .editOrNewMetadata()
    +        .withName(secretName)
    +        .endMetadata()
    +      .addToStringData(TEST_SECRET_KEY, TEST_SECRET_VALUE)
    +      .done()
    +  }
    +
    +  private def checkTestSecret(
    +      secretName: String,
    +      pod: Pod,
    +      withInitContainer: Boolean = false): Unit = {
    +    val testSecretVolume = pod.getSpec.getVolumes.asScala.filter { volume =>
    +      volume.getName == s"$secretName-volume"
    +    }
    +    assert(testSecretVolume.size === 1)
    +    assert(testSecretVolume.head.getSecret.getSecretName === secretName)
    +
    +    checkTestSecretInContainer(secretName, pod.getSpec.getContainers.get(0))
    +
    +    if (withInitContainer) {
    +      checkTestSecretInContainer(secretName, pod.getSpec.getInitContainers.get(0))
    +    }
    +  }
    +
    +  private def checkTestSecretInContainer(secretName: String, container: Container): Unit = {
    +    val testSecret = container.getVolumeMounts.asScala.filter { mount =>
    +      mount.getName == s"$secretName-volume"
    +    }
    +    assert(testSecret.size === 1)
    +    assert(testSecret.head.getMountPath === TEST_SECRET_MOUNT_PATH)
    +  }
    +
    +  private def deleteTestSecret(secretName: String): Unit = {
    +    kubernetesTestComponents.kubernetesClient.secrets
    +      .withName(secretName)
    +      .delete()
    +
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      assert(kubernetesTestComponents.kubernetesClient.secrets.withName(secretName).get() == null)
    +    }
    +  }
    +}
    +
    +private[spark] object KubernetesSuite {
    +
    +  val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
    +  val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
    +  val SPARK_PI_MAIN_CLASS: String = "org.apache.spark.examples.SparkPi"
    +  val SPARK_PAGE_RANK_MAIN_CLASS: String = "org.apache.spark.examples.SparkPageRank"
    +
    +  val TEST_SECRET_NAME_PREFIX = "test-secret-"
    +  val TEST_SECRET_KEY = "test-key"
    +  val TEST_SECRET_VALUE = "test-data"
    +  val TEST_SECRET_MOUNT_PATH = "/etc/secrets"
    +
    +  val CONTAINER_LOCAL_FILE_DOWNLOAD_PATH = "/var/spark-data/spark-files"
    --- End diff --
    
    Removed, thanks!


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91261 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91261/testReport)** for PR 20697 at commit [`901edb3`](https://github.com/apache/spark/commit/901edb3ba3a566e5c6737d15e197950abce5131c).
     * 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 pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194150705
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a very long application name.") {
    +    sparkAppConf.set("spark.app.name", "long" * 40)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a master URL without a scheme.") {
    +    val url = kubernetesTestComponents.kubernetesClient.getMasterUrl
    +    val k8sMasterUrl = if (url.getPort < 0) {
    +      s"k8s://${url.getHost}"
    +    } else {
    +      s"k8s://${url.getHost}:${url.getPort}"
    +    }
    +    sparkAppConf.set("spark.master", k8sMasterUrl)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with an argument.") {
    +    runSparkPiAndVerifyCompletion(appArgs = Array("5"))
    +  }
    +
    +  test("Run SparkPi with custom labels, annotations, and environment variables.") {
    +    sparkAppConf
    +      .set("spark.kubernetes.driver.label.label1", "label1-value")
    +      .set("spark.kubernetes.driver.label.label2", "label2-value")
    +      .set("spark.kubernetes.driver.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.driver.annotation.annotation2", "annotation2-value")
    +      .set("spark.kubernetes.driverEnv.ENV1", "VALUE1")
    +      .set("spark.kubernetes.driverEnv.ENV2", "VALUE2")
    +      .set("spark.kubernetes.executor.label.label1", "label1-value")
    +      .set("spark.kubernetes.executor.label.label2", "label2-value")
    +      .set("spark.kubernetes.executor.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.executor.annotation.annotation2", "annotation2-value")
    +      .set("spark.executorEnv.ENV1", "VALUE1")
    +      .set("spark.executorEnv.ENV2", "VALUE2")
    +
    +    runSparkPiAndVerifyCompletion(
    +      driverPodChecker = (driverPod: Pod) => {
    +        doBasicDriverPodCheck(driverPod)
    +        checkCustomSettings(driverPod)
    +      },
    +      executorPodChecker = (executorPod: Pod) => {
    +        doBasicExecutorPodCheck(executorPod)
    +        checkCustomSettings(executorPod)
    +      })
    +  }
    +
    +  // TODO(ssuchter): Enable the below after debugging
    +  // test("Run PageRank using remote data file") {
    +  //   sparkAppConf
    +  //     .set("spark.kubernetes.mountDependencies.filesDownloadDir",
    +  //       CONTAINER_LOCAL_FILE_DOWNLOAD_PATH)
    +  //     .set("spark.files", REMOTE_PAGE_RANK_DATA_FILE)
    +  //   runSparkPageRankAndVerifyCompletion(
    +  //     appArgs = Array(CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE))
    +  // }
    +
    +  private def runSparkPiAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String] = Array.empty[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PI_MAIN_CLASS,
    +      Seq("Pi is roughly 3"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkPageRankAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PAGE_RANK_MAIN_CLASS,
    +      Seq("1 has rank", "2 has rank", "3 has rank", "4 has rank"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkApplicationAndVerifyCompletion(
    +      appResource: String,
    +      mainClass: String,
    +      expectedLogOnCompletion: Seq[String],
    +      appArgs: Array[String],
    +      driverPodChecker: Pod => Unit,
    +      executorPodChecker: Pod => Unit,
    +      appLocator: String): Unit = {
    +    val appArguments = SparkAppArguments(
    +      mainAppResource = appResource,
    +      mainClass = mainClass,
    +      appArgs = appArgs)
    +    SparkAppLauncher.launch(appArguments, sparkAppConf, TIMEOUT.value.toSeconds.toInt, sparkHomeDir)
    +
    +    val driverPod = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "driver")
    +      .list()
    +      .getItems
    +      .get(0)
    +    driverPodChecker(driverPod)
    +
    +    val executorPods = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "executor")
    +      .list()
    +      .getItems
    +    executorPods.asScala.foreach { pod =>
    +      executorPodChecker(pod)
    +    }
    +
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      expectedLogOnCompletion.foreach { e =>
    +        assert(kubernetesTestComponents.kubernetesClient
    +          .pods()
    +          .withName(driverPod.getMetadata.getName)
    +          .getLog
    +          .contains(e), "The application did not complete.")
    +      }
    +    }
    +  }
    +
    +  private def doBasicDriverPodCheck(driverPod: Pod): Unit = {
    +    assert(driverPod.getMetadata.getName === driverPodName)
    +    assert(driverPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver")
    +  }
    +
    +  private def doBasicExecutorPodCheck(executorPod: Pod): Unit = {
    +    assert(executorPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(executorPod.getSpec.getContainers.get(0).getName === "executor")
    +  }
    +
    +  private def checkCustomSettings(pod: Pod): Unit = {
    +    assert(pod.getMetadata.getLabels.get("label1") === "label1-value")
    +    assert(pod.getMetadata.getLabels.get("label2") === "label2-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation1") === "annotation1-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation2") === "annotation2-value")
    +
    +    val container = pod.getSpec.getContainers.get(0)
    +    val envVars = container
    +      .getEnv
    +      .asScala
    +      .map { env =>
    +        (env.getName, env.getValue)
    +      }
    +      .toMap
    +    assert(envVars("ENV1") === "VALUE1")
    +    assert(envVars("ENV2") === "VALUE2")
    +  }
    +
    +  private def deleteDriverPod(): Unit = {
    +    kubernetesTestComponents.kubernetesClient.pods().withName(driverPodName).delete()
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      assert(kubernetesTestComponents.kubernetesClient
    +        .pods()
    +        .withName(driverPodName)
    +        .get() == null)
    +    }
    +  }
    +}
    +
    +private[spark] object KubernetesSuite {
    +
    +  val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
    +  val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
    +  val SPARK_PI_MAIN_CLASS: String = "org.apache.spark.examples.SparkPi"
    +  val SPARK_PAGE_RANK_MAIN_CLASS: String = "org.apache.spark.examples.SparkPageRank"
    +
    +  // val CONTAINER_LOCAL_FILE_DOWNLOAD_PATH = "/var/spark-data/spark-files"
    +
    +  // val REMOTE_PAGE_RANK_DATA_FILE =
    +  //   "https://storage.googleapis.com/spark-k8s-integration-tests/files/pagerank_data.txt"
    +  // val CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE =
    +  //   s"$CONTAINER_LOCAL_FILE_DOWNLOAD_PATH/pagerank_data.txt"
    +
    +  case object ShuffleNotReadyException extends Exception
    --- End diff --
    
    I commented this out because it's currently unused but will be used soon.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3475/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1348/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3626/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    > so...  yeah.  i'll have a much, much, much better idea of when we'll be
    > getting more ubuntu workers on-line as well as ubuntu-friendly (or fully
    > containerized) spark tests within the next 3-4 weeks.
    >
    
    sorry, let me clarify this a bit:
    
    our "best case" scenario will have has all spark builds, except for the
    PRB, running in containers.
    
    shane



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    > In terms of integration with the dev/run-tests.py, we at most want to make it optional.
    
    It's fine to make them optional; in fact that would be a requirement. The trick would be to run them automatically when there are interesting changes (e.g. changes in the k8s module), and do that as part of the normal PR build, instead of having a separate job that runs k8s integration tests.
    
    I'm not sure what's the current distribution of hosts (Shane can comment on that). but if they all can run k8s integration tests, then I think that would be a better solution than having two separate jobs.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191961749
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    Also - Jenkins has *already* been doing that - building the distribution .tgz for each Kubernetes related PRB invokation. Relevant is the fact that the filtering of what is a Kubernetes-related change happens *before* the distribution .tgz is done.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91168 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91168/testReport)** for PR 20697 at commit [`aa72b6e`](https://github.com/apache/spark/commit/aa72b6e947b588d834ed4840f24c3b3483f73d9f).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194128800
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.Closeable
    +import java.net.URI
    +
    +import org.apache.spark.internal.Logging
    +
    +object Utils extends Logging {
    +
    +  def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = {
    +    val resource = createResource
    +    try f.apply(resource) finally resource.close()
    +  }
    +
    +  def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = {
    +    var originalThrowable: Throwable = null
    +    try {
    +      block
    +    } catch {
    +      case t: Throwable =>
    +        // Purposefully not using NonFatal, because even fatal exceptions
    +        // we don't want to have our finallyBlock suppress
    +        originalThrowable = t
    +        throw originalThrowable
    +    } finally {
    +      try {
    +        finallyBlock
    +      } catch {
    +        case t: Throwable =>
    +          if (originalThrowable != null) {
    +            originalThrowable.addSuppressed(t)
    +            logWarning(s"Suppressing exception in finally: " + t.getMessage, t)
    +            throw originalThrowable
    +          } else {
    +            throw t
    +          }
    +      }
    +    }
    +  }
    +
    +  def checkAndGetK8sMasterUrl(rawMasterURL: String): String = {
    --- End diff --
    
    Similarly for this method - don't think it's used


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Hm, I'm glad I tried to paste output of running against master (well, strictly my rebase of this PR, but functionally the same thing). It's throwing some error that I need to debug:
    
      2018-05-26 00:46:32 INFO  KubernetesClusterSchedulerBackend:54 - SchedulerBackend is ready for scheduling beginning after reached minRegisteredResourcesRatio: 0.8
      Exception in thread "main" java.lang.NumberFormatException: For input string: ""
      	at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
      	at java.lang.Integer.parseInt(Integer.java:592)
      	at java.lang.Integer.parseInt(Integer.java:615)
      	at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
      	at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
      	at org.apache.spark.examples.SparkPi$.main(SparkPi.scala:32)
      	at org.apache.spark.examples.SparkPi.main(SparkPi.scala)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
      	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:840)
      	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167)
      	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:194)
      	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
      	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:915)
      	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:926)
      	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    we tried the --vm-driver=none and it wasn't working for us...  this was ~8
    months ago, however, and i can't recall exactly went wrong.
    
    On Fri, Jun 1, 2018 at 10:49 AM, Sean Suchter <no...@github.com>
    wrote:
    
    > @skonto <https://github.com/skonto> I'll test that and discuss with
    > @shaneknapp <https://github.com/shaneknapp>. It wouldn't involve directly
    > changing code in this PR, since the minikube creation/destruction is done
    > by Jenkins job config, but it is relevant to how we set up systems. We
    > could potentially accelerate the adoption of all nodes. I'm not sure about
    > whether the docker system is ready on all the nodes or not, but that's a
    > good discussion.
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/20697#issuecomment-393958574>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/ABiDrCMEvK9wFwx_JFAcMagoK_yd5kDtks5t4X7BgaJpZM4SXRkC>
    > .
    >



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91406/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191962980
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    I timed the build on my laptop. To build the Spark jars took just over 11 minutes. To build the .tgz took about 7 seconds. So this extra step adds ~1% overhead.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    just wanted to chime in here:
    
    i went to databricks a couple of weeks back and have confirmation that
    after the spark summit, that i will officially have build system support
    from them!  this took a lot of time for me to wrangle, but it's finally
    happened.  :)
    
    the biggest win for us here is that getting spark builds ported to ubuntu
    in the next couple of months.  this means that we'll have minikube
    installed on all of the workers.
    
    so...  yeah.  i'll have a much, much, much better idea of when we'll be
    getting more ubuntu workers on-line as well as ubuntu-friendly (or fully
    containerized) spark tests within the next 3-4 weeks.
    
    shane
    
    On Fri, May 25, 2018 at 2:16 PM, Sean Suchter <no...@github.com>
    wrote:
    
    > Hm, that wasn't the answer I expected to hear. It leads me to another
    > question:
    >
    > If we don't have developers using this script directly, what's the value
    > of trying to incorporate the running of the Kubernetes integration tests
    > into this program? Because of the AMPLab setup, we're going to have to have
    > special Jenkins jobs dedicated to the running of the K8s integration tests.
    > This is because not all hosts can run these integration tests, so therefore
    > we have special jobs that are tied to the specific hosts that have the
    > right setup to run the minikube VMs. So it's just going to be those jobs
    > that call dev/run-tests.py with the special non-default argument to run
    > these tests. Those jenkins jobs could just as easily run a different script.
    >
    > An alternative way to take my argument is that we should delay the
    > integration into run-tests.py until after *all* of the AMPLab hosts can
    > run these tests. Since that's not today, it wouldn't need to be in this
    > initial PR, and could have its own PR.
    >
    > Thoughts? Thanks, Sean
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/20697#issuecomment-392189209>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/ABiDrFJdW_7DLsOILqAJ1ENZyZml4tc4ks5t2HSxgaJpZM4SXRkC>
    > .
    >



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91264 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91264/testReport)** for PR 20697 at commit [`dd28032`](https://github.com/apache/spark/commit/dd280327ea0cec6d1643007679d189529c4bc1db).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @vanzin @felixcheung I'm trying to look into this again. My main question is how people use this dev/run-tests.py suite. Every time I run it, even though I've only changed things in the kubernetes resourcemanager section, it takes hours to run, and isn't even running the kubernetes tests. Is there some command line flags or ways that people use this tool to just select the tests that they want to run? Please feel free to point me at a doc to read, I'm just not sure which one to read.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91173 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91173/testReport)** for PR 20697 at commit [`5ffa464`](https://github.com/apache/spark/commit/5ffa464c65d9caaae86df059a75f99a893acdcf9).
     * 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 issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3598/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194150614
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.Closeable
    +import java.net.URI
    +
    +import org.apache.spark.internal.Logging
    +
    +object Utils extends Logging {
    +
    +  def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = {
    +    val resource = createResource
    +    try f.apply(resource) finally resource.close()
    +  }
    +
    +  def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = {
    --- End diff --
    
    I agree. I removed it. I did a series of greps and removed things that were't used.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Hi folks - I don't think there is any more work to do on this PR. Is there something you're waiting for before merging it?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3859/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3476/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191026951
  
    --- Diff: resource-managers/kubernetes/integration-tests/e2e/e2e-prow.sh ---
    @@ -0,0 +1,69 @@
    +#!/bin/bash
    +
    +# 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.
    +
    +### This script is used by Kubernetes Test Infrastructure to run integration tests.
    +### See documenation at https://github.com/kubernetes/test-infra/tree/master/prow
    +
    +set -ex
    +
    +# set cwd correctly
    +cd "$(dirname "$0")/../"
    +
    +# Include requisite scripts
    +source ./include/util.sh
    +
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +BRANCH="master"
    +SPARK_REPO="https://github.com/apache/spark"
    +SPARK_REPO_LOCAL_DIR="$TEST_ROOT_DIR/target/spark"
    +
    +## Install basic dependencies
    +## These are for the kubekins-e2e environment in https://github.com/kubernetes/test-infra/tree/master/images/kubekins-e2e
    +echo "deb http://http.debian.net/debian jessie-backports main" >> /etc/apt/sources.list
    +apt-get update && apt-get install -y curl wget git tar uuid-runtime
    --- End diff --
    
    ping - probably don't want this stuff here yet.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3598/



---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191629581
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    I'm sure it's possible. But I'd rather not try to do that refactor. I'd be really happy if you wanted to.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91406 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91406/testReport)** for PR 20697 at commit [`4c5677a`](https://github.com/apache/spark/commit/4c5677a61fd940b818d81469e6640cb45f00ce58).


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91351 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91351/testReport)** for PR 20697 at commit [`67cb6e2`](https://github.com/apache/spark/commit/67cb6e21cd4726ae398c5f0045b1d758fb6a8238).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r194133234
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a very long application name.") {
    +    sparkAppConf.set("spark.app.name", "long" * 40)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a master URL without a scheme.") {
    +    val url = kubernetesTestComponents.kubernetesClient.getMasterUrl
    +    val k8sMasterUrl = if (url.getPort < 0) {
    +      s"k8s://${url.getHost}"
    +    } else {
    +      s"k8s://${url.getHost}:${url.getPort}"
    +    }
    +    sparkAppConf.set("spark.master", k8sMasterUrl)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with an argument.") {
    +    runSparkPiAndVerifyCompletion(appArgs = Array("5"))
    +  }
    +
    +  test("Run SparkPi with custom labels, annotations, and environment variables.") {
    +    sparkAppConf
    +      .set("spark.kubernetes.driver.label.label1", "label1-value")
    +      .set("spark.kubernetes.driver.label.label2", "label2-value")
    +      .set("spark.kubernetes.driver.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.driver.annotation.annotation2", "annotation2-value")
    +      .set("spark.kubernetes.driverEnv.ENV1", "VALUE1")
    +      .set("spark.kubernetes.driverEnv.ENV2", "VALUE2")
    +      .set("spark.kubernetes.executor.label.label1", "label1-value")
    +      .set("spark.kubernetes.executor.label.label2", "label2-value")
    +      .set("spark.kubernetes.executor.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.executor.annotation.annotation2", "annotation2-value")
    +      .set("spark.executorEnv.ENV1", "VALUE1")
    +      .set("spark.executorEnv.ENV2", "VALUE2")
    +
    +    runSparkPiAndVerifyCompletion(
    +      driverPodChecker = (driverPod: Pod) => {
    +        doBasicDriverPodCheck(driverPod)
    +        checkCustomSettings(driverPod)
    +      },
    +      executorPodChecker = (executorPod: Pod) => {
    +        doBasicExecutorPodCheck(executorPod)
    +        checkCustomSettings(executorPod)
    +      })
    +  }
    +
    +  // TODO(ssuchter): Enable the below after debugging
    --- End diff --
    
    I don't know why it's broken, but @mccheah and I agreed that fixing it doesn't have to be in this PR.


---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    - Bug #?
    - Proper title: [SPARK-xxxxx][k8s] Blah.
    - Why isn't this hooked up to the build (both mvn and sbt)?
    - You have redundant stuff like a license file and a mvn script
    
    After this is cleaned up and ready to get in, I'd recommend integrating the tests into `dev/run-tests.py` instead of running a separate job just for this. We should work with the amplab folks to make sure that the VMs used by the regular PR builder can also run these tests.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1164/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191957294
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    Would the Jenkins framework tied into Spark pull requests have to build the tgz then? I expect that would create a non-trivial amount of overhead on each build.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3596/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191961317
  
    --- Diff: resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh ---
    @@ -0,0 +1,91 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked"
    +IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt"
    +DEPLOY_MODE="minikube"
    +IMAGE_REPO="docker.io/kubespark"
    +IMAGE_TAG="N/A"
    +SPARK_TGZ="N/A"
    +
    +# Parse arguments
    +while (( "$#" )); do
    +  case $1 in
    +    --unpacked-spark-tgz)
    +      UNPACKED_SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    --image-repo)
    +      IMAGE_REPO="$2"
    +      shift
    +      ;;
    +    --image-tag)
    +      IMAGE_TAG="$2"
    +      shift
    +      ;;
    +    --image-tag-output-file)
    +      IMAGE_TAG_OUTPUT_FILE="$2"
    +      shift
    +      ;;
    +    --deploy-mode)
    +      DEPLOY_MODE="$2"
    +      shift
    +      ;;
    +    --spark-tgz)
    +      SPARK_TGZ="$2"
    +      shift
    +      ;;
    +    *)
    +      break
    +      ;;
    +  esac
    +  shift
    +done
    +
    +if [[ $SPARK_TGZ == "N/A" ]];
    +then
    +  echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1;
    --- End diff --
    
    The act of building the .tgz is much cheaper (faster) than doing the java build or the integration test. I wouldn't worry about that.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191028204
  
    --- Diff: resource-managers/kubernetes/integration-tests/include/util.sh ---
    @@ -0,0 +1,43 @@
    +#!/usr/bin/env bash
    +
    +# 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.
    +
    +clone_build_spark() {
    --- End diff --
    
    Removed, thanks


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I'm ok with this. I'm guessing we'd then have to switch the K8s integration test PRB to pull the repository and use this repository itself to invoke the test scripts, yeah? And then we no longer reference the old k8s spark-integration repository?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3474/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1867/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    (FYI - SparkQA Kubernetes integration test != SparkQA Test build)


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191034109
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.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.k8s.integrationtest
    +
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import io.fabric8.kubernetes.client.DefaultKubernetesClient
    +import org.scalatest.concurrent.Eventually
    +
    +private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) {
    +
    +  val namespaceOption = Option(System.getProperty("spark.kubernetes.test.namespace"))
    +  val hasUserSpecifiedNamespace = namespaceOption.isDefined
    +  val namespace = namespaceOption.getOrElse(UUID.randomUUID().toString.replaceAll("-", ""))
    +  private val serviceAccountName =
    +    Option(System.getProperty("spark.kubernetes.test.serviceAccountName"))
    +      .getOrElse("default")
    +  val kubernetesClient = defaultClient.inNamespace(namespace)
    +  val clientConfig = kubernetesClient.getConfiguration
    +
    +  def createNamespace(): Unit = {
    +    defaultClient.namespaces.createNew()
    +      .withNewMetadata()
    +      .withName(namespace)
    +      .endMetadata()
    +      .done()
    +  }
    +
    +  def deleteNamespace(): Unit = {
    +    defaultClient.namespaces.withName(namespace).delete()
    +    Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
    +      val namespaceList = defaultClient
    +        .namespaces()
    +        .list()
    +        .getItems
    +        .asScala
    +      require(!namespaceList.exists(_.getMetadata.getName == namespace))
    +    }
    +  }
    +
    +  def newSparkAppConf(): SparkAppConf = {
    +    new SparkAppConf()
    +      .set("spark.master", s"k8s://${kubernetesClient.getMasterUrl}")
    +      .set("spark.kubernetes.namespace", namespace)
    +      .set("spark.executor.memory", "500m")
    +      .set("spark.executor.cores", "1")
    +      .set("spark.executors.instances", "1")
    +      .set("spark.app.name", "spark-test-app")
    +      .set("spark.ui.enabled", "true")
    +      .set("spark.testing", "false")
    +      .set("spark.kubernetes.submission.waitAppCompletion", "false")
    +      .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName)
    +  }
    +}
    +
    +private[spark] class SparkAppConf {
    +
    +  private val map = mutable.Map[String, String]()
    +
    +  def set(key: String, value: String): SparkAppConf = {
    +    map.put(key, value)
    +    this
    +  }
    +
    +  def get(key: String): String = map.getOrElse(key, "")
    +
    +  def setJars(jars: Seq[String]): Unit = set("spark.jars", jars.mkString(","))
    +
    +  override def toString: String = map.toString
    +
    +  def toStringArray: Iterable[String] = map.toList.flatMap(t => List("--conf", s"${t._1}=${t._2}"))
    +}
    +
    +private[spark] case class SparkAppArguments(
    +    mainAppResource: String,
    +    mainClass: String,
    +    appArgs: Array[String])
    +
    +private[spark] object SparkAppLauncher extends Logging {
    --- End diff --
    
    An idea is to use `SparkLauncher` but for a first iteration it's fine to not do that.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3476/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191033936
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    +    <commons-logging.version>1.1.1</commons-logging.version>
    +    <docker-client.version>5.0.2</docker-client.version>
    +    <download-maven-plugin.version>1.3.0</download-maven-plugin.version>
    +    <exec-maven-plugin.version>1.4.0</exec-maven-plugin.version>
    +    <extraScalaTestArgs></extraScalaTestArgs>
    +    <guava.version>18.0</guava.version>
    +    <jsr305.version>1.3.9</jsr305.version>
    +    <kubernetes-client.version>3.0.0</kubernetes-client.version>
    +    <log4j.version>1.2.17</log4j.version>
    +    <scala.version>2.11.8</scala.version>
    +    <scala.binary.version>2.11</scala.binary.version>
    +    <scala-maven-plugin.version>3.2.2</scala-maven-plugin.version>
    +    <scalatest.version>2.2.6</scalatest.version>
    +    <scalatest-maven-plugin.version>1.0</scalatest-maven-plugin.version>
    +    <slf4j-log4j12.version>1.7.24</slf4j-log4j12.version>
    +    <sbt.project.name>kubernetes-integration-tests</sbt.project.name>
    +    <spark.kubernetes.test.unpackSparkDir>${project.build.directory}/spark-dist-unpacked</spark.kubernetes.test.unpackSparkDir>
    +    <spark.kubernetes.test.imageTag>N/A</spark.kubernetes.test.imageTag>
    +    <spark.kubernetes.test.imageTagFile>${project.build.directory}/imageTag.txt</spark.kubernetes.test.imageTagFile>
    +    <spark.kubernetes.test.deployMode>minikube</spark.kubernetes.test.deployMode>
    +    <spark.kubernetes.test.imageRepo>docker.io/kubespark</spark.kubernetes.test.imageRepo>
    +    <test.exclude.tags></test.exclude.tags>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes Integration Tests</name>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>commons-logging</groupId>
    +      <artifactId>commons-logging</artifactId>
    +      <version>${commons-logging.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.code.findbugs</groupId>
    +      <artifactId>jsr305</artifactId>
    +      <version>${jsr305.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <scope>test</scope>
    +      <!-- For compatibility with Docker client. Should be fine since this is just for tests.-->
    +      <version>${guava.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.spotify</groupId>
    +      <artifactId>docker-client</artifactId>
    +      <version>${docker-client.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    +      <version>${kubernetes-client.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>log4j</groupId>
    +      <artifactId>log4j</artifactId>
    +      <version>${log4j.version}</version>
    --- End diff --
    
    ...and all these versions should be provided by the parent pom too.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Ok, I think all open issues have been resolved. The PRB failures are because of github request failures, so they are spurious. @vanzin @mccheah I think it's ready for another look.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3472/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3471/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @ssuchter I think minikube could run within docker with flag –vm-driver=none right? This way you dont need minikube on hosts. Just an idea havent tried might be wrong.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191483976
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    +    <commons-logging.version>1.1.1</commons-logging.version>
    +    <docker-client.version>5.0.2</docker-client.version>
    +    <download-maven-plugin.version>1.3.0</download-maven-plugin.version>
    +    <exec-maven-plugin.version>1.4.0</exec-maven-plugin.version>
    +    <extraScalaTestArgs></extraScalaTestArgs>
    +    <guava.version>18.0</guava.version>
    +    <jsr305.version>1.3.9</jsr305.version>
    +    <kubernetes-client.version>3.0.0</kubernetes-client.version>
    +    <log4j.version>1.2.17</log4j.version>
    +    <scala.version>2.11.8</scala.version>
    +    <scala.binary.version>2.11</scala.binary.version>
    +    <scala-maven-plugin.version>3.2.2</scala-maven-plugin.version>
    +    <scalatest.version>2.2.6</scalatest.version>
    +    <scalatest-maven-plugin.version>1.0</scalatest-maven-plugin.version>
    +    <slf4j-log4j12.version>1.7.24</slf4j-log4j12.version>
    +    <sbt.project.name>kubernetes-integration-tests</sbt.project.name>
    +    <spark.kubernetes.test.unpackSparkDir>${project.build.directory}/spark-dist-unpacked</spark.kubernetes.test.unpackSparkDir>
    +    <spark.kubernetes.test.imageTag>N/A</spark.kubernetes.test.imageTag>
    +    <spark.kubernetes.test.imageTagFile>${project.build.directory}/imageTag.txt</spark.kubernetes.test.imageTagFile>
    +    <spark.kubernetes.test.deployMode>minikube</spark.kubernetes.test.deployMode>
    +    <spark.kubernetes.test.imageRepo>docker.io/kubespark</spark.kubernetes.test.imageRepo>
    +    <test.exclude.tags></test.exclude.tags>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes Integration Tests</name>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>commons-logging</groupId>
    +      <artifactId>commons-logging</artifactId>
    +      <version>${commons-logging.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.code.findbugs</groupId>
    +      <artifactId>jsr305</artifactId>
    +      <version>${jsr305.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <scope>test</scope>
    +      <!-- For compatibility with Docker client. Should be fine since this is just for tests.-->
    +      <version>${guava.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.spotify</groupId>
    +      <artifactId>docker-client</artifactId>
    +      <version>${docker-client.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    +      <version>${kubernetes-client.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>log4j</groupId>
    +      <artifactId>log4j</artifactId>
    +      <version>${log4j.version}</version>
    --- End diff --
    
    Resolved in 901edb3ba3, f68cdba77b, dd280327ea


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/1351/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91576 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91576/testReport)** for PR 20697 at commit [`f67d582`](https://github.com/apache/spark/commit/f67d5823b235e04211a9a8d161a3382e14adda8f).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191487129
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.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.k8s.integrationtest
    +
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import io.fabric8.kubernetes.client.DefaultKubernetesClient
    +import org.scalatest.concurrent.Eventually
    +
    +private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) {
    +
    +  val namespaceOption = Option(System.getProperty("spark.kubernetes.test.namespace"))
    +  val hasUserSpecifiedNamespace = namespaceOption.isDefined
    +  val namespace = namespaceOption.getOrElse(UUID.randomUUID().toString.replaceAll("-", ""))
    +  private val serviceAccountName =
    +    Option(System.getProperty("spark.kubernetes.test.serviceAccountName"))
    +      .getOrElse("default")
    +  val kubernetesClient = defaultClient.inNamespace(namespace)
    +  val clientConfig = kubernetesClient.getConfiguration
    +
    +  def createNamespace(): Unit = {
    +    defaultClient.namespaces.createNew()
    +      .withNewMetadata()
    +      .withName(namespace)
    +      .endMetadata()
    +      .done()
    +  }
    +
    +  def deleteNamespace(): Unit = {
    +    defaultClient.namespaces.withName(namespace).delete()
    +    Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
    +      val namespaceList = defaultClient
    +        .namespaces()
    +        .list()
    +        .getItems
    +        .asScala
    +      require(!namespaceList.exists(_.getMetadata.getName == namespace))
    +    }
    +  }
    +
    +  def newSparkAppConf(): SparkAppConf = {
    +    new SparkAppConf()
    +      .set("spark.master", s"k8s://${kubernetesClient.getMasterUrl}")
    +      .set("spark.kubernetes.namespace", namespace)
    +      .set("spark.executor.memory", "500m")
    +      .set("spark.executor.cores", "1")
    +      .set("spark.executors.instances", "1")
    +      .set("spark.app.name", "spark-test-app")
    +      .set("spark.ui.enabled", "true")
    +      .set("spark.testing", "false")
    +      .set("spark.kubernetes.submission.waitAppCompletion", "false")
    +      .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName)
    +  }
    +}
    +
    +private[spark] class SparkAppConf {
    +
    +  private val map = mutable.Map[String, String]()
    +
    +  def set(key: String, value: String): SparkAppConf = {
    +    map.put(key, value)
    +    this
    +  }
    +
    +  def get(key: String): String = map.getOrElse(key, "")
    +
    +  def setJars(jars: Seq[String]): Unit = set("spark.jars", jars.mkString(","))
    +
    +  override def toString: String = map.toString
    +
    +  def toStringArray: Iterable[String] = map.toList.flatMap(t => List("--conf", s"${t._1}=${t._2}"))
    +}
    +
    +private[spark] case class SparkAppArguments(
    +    mainAppResource: String,
    +    mainClass: String,
    +    appArgs: Array[String])
    +
    +private[spark] object SparkAppLauncher extends Logging {
    --- End diff --
    
    This one is so much smaller (< 10 lines of executable code) than SparkLauncher, I think we should not try to switch in this CL.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91236/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191486261
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties ---
    @@ -0,0 +1,31 @@
    +#
    +# 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.
    +#
    +
    +# Set everything to be logged to the file target/integration-tests.log
    +log4j.rootCategory=INFO, file
    --- End diff --
    
    It's still necessary, it does not inherit.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91265 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91265/testReport)** for PR 20697 at commit [`5a8fd7f`](https://github.com/apache/spark/commit/5a8fd7ff400bcf44e94800905bfa48715eaba88c).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3522/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191028169
  
    --- Diff: resource-managers/kubernetes/integration-tests/e2e/e2e-prow.sh ---
    @@ -0,0 +1,69 @@
    +#!/bin/bash
    +
    +# 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.
    +
    +### This script is used by Kubernetes Test Infrastructure to run integration tests.
    +### See documenation at https://github.com/kubernetes/test-infra/tree/master/prow
    +
    +set -ex
    +
    +# set cwd correctly
    +cd "$(dirname "$0")/../"
    +
    +# Include requisite scripts
    +source ./include/util.sh
    +
    +TEST_ROOT_DIR=$(git rev-parse --show-toplevel)
    +BRANCH="master"
    +SPARK_REPO="https://github.com/apache/spark"
    +SPARK_REPO_LOCAL_DIR="$TEST_ROOT_DIR/target/spark"
    +
    +## Install basic dependencies
    +## These are for the kubekins-e2e environment in https://github.com/kubernetes/test-infra/tree/master/images/kubekins-e2e
    +echo "deb http://http.debian.net/debian jessie-backports main" >> /etc/apt/sources.list
    +apt-get update && apt-get install -y curl wget git tar uuid-runtime
    --- End diff --
    
    Removed, thanks


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Ok, I didn't do anything wrong. There is a bug in the Github web UI that the PR's web view doesn't get updated when I rebase it. I had to change what Github thought the branch I was merging into to 2.3 and then back to master, and now it shows correctly just my diffs.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191033878
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Logging.scala ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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.integrationtest
    +
    +import org.apache.log4j.{Logger, LogManager, Priority}
    +
    +trait Logging {
    --- End diff --
    
    Oh we don't need this, we should be able to just use spark-core's `Logging` trait, yeah?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    @ssuchter the jira ticket for this is https://issues.apache.org/jira/browse/SPARK-23010.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91170 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91170/testReport)** for PR 20697 at commit [`81956aa`](https://github.com/apache/spark/commit/81956aac94dc9b76bc4a2af5dfe9a5db6fe788ef).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3543/



---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r172680337
  
    --- Diff: resource-managers/kubernetes/integration-tests/build/mvn ---
    @@ -0,0 +1,29 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +
    +BUILD_DIR=$(dirname $0)
    --- End diff --
    
    We shouldn't need another `mvn`, we have one already in the Spark repository.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3475/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    @ssuchter good point, they should. I thought they were set up to be invoked because kube has already been added to modules.py


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    You're right, the prow-based codepath still does the cloning. Sorry! I did remove the cloning and building option in the entry point resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh, so it's not all still there.
    
    @mccheah - Why do we need the prow-based codepath at all in this PR? AMPLab jenkins doesn't use it. It only uses resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3581/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I'm not sure why it's looking in the `dist` directory; that is only populated by the build, there's no checked in code there. Maybe run-tests (or whatever code is throwing the error) should be fixed to not look at it.
    
    Or you could clean your repo before running the script.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191483921
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    --- End diff --
    
    Resolved in 901edb3ba3, f68cdba77b, dd280327ea


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191033921
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    --- End diff --
    
    Hm, I'd expect all of these to already be pulled in by the `spark-parent`. I think this is an artifact of translation from a standalone project to being embedded here.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3472/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    > If we don't have developers using this script directly, what's the value of trying to incorporate the running of the Kubernetes integration tests into this program?
    
    Every PR build runs that script. It would avoid having special jobs just for kubernetes testing (which, frankly, as someone who is not working on k8s stuff 99.9% of the time, is super confusing). If not all hosts currently can run them, then it's ok to postpone this. But I still think all PR testing should be, eventually, tied to that script.
    
    For example, release managers generally use that script for testing, and doing so would allow them to easily include kubernetes in their testing without having to jump through hoops.
    
    If you're worried about your own testing, you can follow the python / R approach and have a separate "run-tests" script, and call that one from the main run-tests script.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Ok, I think all open issues have been resolved. The PRB failures are because of github request failures, so they are spurious. @vanzin @mccheah I think it's ready for another look.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3650/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3473/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91176 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91176/testReport)** for PR 20697 at commit [`65347b3`](https://github.com/apache/spark/commit/65347b319a6491071060cdc95923ef65b68aad94).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Oops, I really did something wrong there. Please ignore while I fix.


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Thanks @mccheah and @ssuchter! Added a deprecation notice to the repo in https://github.com/apache-spark-on-k8s/spark-integration


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Yeah, I just figured that out. (That it's because of the dist/ directory.) I'll change dev/tox.ini so I don't have to keep cleaning. Thanks!


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1357/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191485369
  
    --- Diff: resource-managers/kubernetes/integration-tests/pom.xml ---
    @@ -0,0 +1,230 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +  ~ 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.
    +  -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <parent>
    +    <groupId>org.apache.spark</groupId>
    +    <artifactId>spark-parent_2.11</artifactId>
    +    <version>2.4.0-SNAPSHOT</version>
    +    <relativePath>../../../pom.xml</relativePath>
    +  </parent>
    +
    +  <artifactId>spark-kubernetes-integration-tests_2.11</artifactId>
    +  <groupId>spark-kubernetes-integration-tests</groupId>
    +  <properties>
    +    <maven.version>3.3.9</maven.version>
    +    <commons-lang3.version>3.5</commons-lang3.version>
    +    <commons-logging.version>1.1.1</commons-logging.version>
    +    <docker-client.version>5.0.2</docker-client.version>
    +    <download-maven-plugin.version>1.3.0</download-maven-plugin.version>
    +    <exec-maven-plugin.version>1.4.0</exec-maven-plugin.version>
    +    <extraScalaTestArgs></extraScalaTestArgs>
    +    <guava.version>18.0</guava.version>
    +    <jsr305.version>1.3.9</jsr305.version>
    +    <kubernetes-client.version>3.0.0</kubernetes-client.version>
    +    <log4j.version>1.2.17</log4j.version>
    +    <scala.version>2.11.8</scala.version>
    +    <scala.binary.version>2.11</scala.binary.version>
    +    <scala-maven-plugin.version>3.2.2</scala-maven-plugin.version>
    +    <scalatest.version>2.2.6</scalatest.version>
    +    <scalatest-maven-plugin.version>1.0</scalatest-maven-plugin.version>
    +    <slf4j-log4j12.version>1.7.24</slf4j-log4j12.version>
    +    <sbt.project.name>kubernetes-integration-tests</sbt.project.name>
    +    <spark.kubernetes.test.unpackSparkDir>${project.build.directory}/spark-dist-unpacked</spark.kubernetes.test.unpackSparkDir>
    +    <spark.kubernetes.test.imageTag>N/A</spark.kubernetes.test.imageTag>
    +    <spark.kubernetes.test.imageTagFile>${project.build.directory}/imageTag.txt</spark.kubernetes.test.imageTagFile>
    +    <spark.kubernetes.test.deployMode>minikube</spark.kubernetes.test.deployMode>
    +    <spark.kubernetes.test.imageRepo>docker.io/kubespark</spark.kubernetes.test.imageRepo>
    +    <test.exclude.tags></test.exclude.tags>
    +  </properties>
    +  <packaging>jar</packaging>
    +  <name>Spark Project Kubernetes Integration Tests</name>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>org.apache.spark</groupId>
    +      <artifactId>spark-core_${scala.binary.version}</artifactId>
    +      <version>${project.version}</version>
    +      <type>test-jar</type>
    +      <scope>test</scope>
    +    </dependency>
    +
    +    <dependency>
    +      <groupId>commons-logging</groupId>
    +      <artifactId>commons-logging</artifactId>
    +      <version>${commons-logging.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.code.findbugs</groupId>
    +      <artifactId>jsr305</artifactId>
    +      <version>${jsr305.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <scope>test</scope>
    +      <!-- For compatibility with Docker client. Should be fine since this is just for tests.-->
    +      <version>${guava.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.spotify</groupId>
    +      <artifactId>docker-client</artifactId>
    +      <version>${docker-client.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>io.fabric8</groupId>
    +      <artifactId>kubernetes-client</artifactId>
    +      <version>${kubernetes-client.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>log4j</groupId>
    +      <artifactId>log4j</artifactId>
    +      <version>${log4j.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.apache.commons</groupId>
    +      <artifactId>commons-lang3</artifactId>
    +      <version>${commons-lang3.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.scala-lang</groupId>
    +      <artifactId>scala-library</artifactId>
    +      <version>${scala.version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.scalatest</groupId>
    +      <artifactId>scalatest_${scala.binary.version}</artifactId>
    +      <scope>test</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.slf4j</groupId>
    +      <artifactId>slf4j-log4j12</artifactId>
    +      <version>${slf4j-log4j12.version}</version>
    +      <scope>test</scope>
    +    </dependency>
    +  </dependencies>
    +
    +  <build>
    +    <plugins>
    +      <plugin>
    +        <groupId>net.alchim31.maven</groupId>
    --- End diff --
    
    Resolved in 5a8fd7ff40


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Thanks for the pointer about @ifilonenko's comment. I removed the Kerberos/secrets related code. I'll post the output momentarily.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91320 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91320/testReport)** for PR 20697 at commit [`4102b25`](https://github.com/apache/spark/commit/4102b25537288fd4d6cf8267f07c52a79f49dd72).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91175 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91175/testReport)** for PR 20697 at commit [`dbce275`](https://github.com/apache/spark/commit/dbce27578439528ab699945f724c0c4c2888c985).
     * 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697#discussion_r171426248
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,391 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a very long application name.") {
    +    sparkAppConf.set("spark.app.name", "long" * 40)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with a master URL without a scheme.") {
    +    val url = kubernetesTestComponents.kubernetesClient.getMasterUrl
    +    val k8sMasterUrl = if (url.getPort < 0) {
    +      s"k8s://${url.getHost}"
    +    } else {
    +      s"k8s://${url.getHost}:${url.getPort}"
    +    }
    +    sparkAppConf.set("spark.master", k8sMasterUrl)
    +    runSparkPiAndVerifyCompletion()
    +  }
    +
    +  test("Run SparkPi with an argument.") {
    +    runSparkPiAndVerifyCompletion(appArgs = Array("5"))
    +  }
    +
    +  test("Run SparkPi with custom labels, annotations, and environment variables.") {
    +    sparkAppConf
    +      .set("spark.kubernetes.driver.label.label1", "label1-value")
    +      .set("spark.kubernetes.driver.label.label2", "label2-value")
    +      .set("spark.kubernetes.driver.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.driver.annotation.annotation2", "annotation2-value")
    +      .set("spark.kubernetes.driverEnv.ENV1", "VALUE1")
    +      .set("spark.kubernetes.driverEnv.ENV2", "VALUE2")
    +      .set("spark.kubernetes.executor.label.label1", "label1-value")
    +      .set("spark.kubernetes.executor.label.label2", "label2-value")
    +      .set("spark.kubernetes.executor.annotation.annotation1", "annotation1-value")
    +      .set("spark.kubernetes.executor.annotation.annotation2", "annotation2-value")
    +      .set("spark.executorEnv.ENV1", "VALUE1")
    +      .set("spark.executorEnv.ENV2", "VALUE2")
    +
    +    runSparkPiAndVerifyCompletion(
    +      driverPodChecker = (driverPod: Pod) => {
    +        doBasicDriverPodCheck(driverPod)
    +        checkCustomSettings(driverPod)
    +      },
    +      executorPodChecker = (executorPod: Pod) => {
    +        doBasicExecutorPodCheck(executorPod)
    +        checkCustomSettings(executorPod)
    +      })
    +  }
    +
    +  test("Run SparkPi with a test secret mounted into the driver and executor pods") {
    +    val secretName = TEST_SECRET_NAME_PREFIX + UUID.randomUUID().toString.replaceAll("-", "")
    +    createTestSecret(secretName)
    +
    +    sparkAppConf
    +      .set(s"spark.kubernetes.driver.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +      .set(s"spark.kubernetes.executor.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +
    +    try {
    +      runSparkPiAndVerifyCompletion(
    +        driverPodChecker = (driverPod: Pod) => {
    +          doBasicDriverPodCheck(driverPod)
    +          checkTestSecret(secretName, driverPod)
    +        },
    +        executorPodChecker = (executorPod: Pod) => {
    +          doBasicExecutorPodCheck(executorPod)
    +          checkTestSecret(secretName, executorPod)
    +        })
    +    } finally {
    +      deleteTestSecret(secretName)
    +    }
    +  }
    +
    +  test("Run PageRank using remote data file") {
    +    sparkAppConf
    +      .set("spark.kubernetes.mountDependencies.filesDownloadDir",
    +        CONTAINER_LOCAL_FILE_DOWNLOAD_PATH)
    +      .set("spark.files", REMOTE_PAGE_RANK_DATA_FILE)
    +    runSparkPageRankAndVerifyCompletion(
    +      appArgs = Array(CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE))
    +  }
    +
    +  test("Run PageRank using remote data file with test secret mounted into the driver and " +
    +    "executors") {
    +    val secretName = TEST_SECRET_NAME_PREFIX + UUID.randomUUID().toString.replaceAll("-", "")
    +    createTestSecret(secretName)
    +
    +    sparkAppConf
    +      .set("spark.kubernetes.mountDependencies.filesDownloadDir",
    +        CONTAINER_LOCAL_FILE_DOWNLOAD_PATH)
    +      .set("spark.files", REMOTE_PAGE_RANK_DATA_FILE)
    +      .set(s"spark.kubernetes.driver.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +      .set(s"spark.kubernetes.executor.secrets.$secretName", TEST_SECRET_MOUNT_PATH)
    +
    +    try {
    +      runSparkPageRankAndVerifyCompletion(
    +        appArgs = Array(CONTAINER_LOCAL_DOWNLOADED_PAGE_RANK_DATA_FILE),
    +        driverPodChecker = (driverPod: Pod) => {
    +          doBasicDriverPodCheck(driverPod)
    +          checkTestSecret(secretName, driverPod, withInitContainer = true)
    +        },
    +        executorPodChecker = (executorPod: Pod) => {
    +          doBasicExecutorPodCheck(executorPod)
    +          checkTestSecret(secretName, executorPod, withInitContainer = true)
    +        })
    +    } finally {
    +      deleteTestSecret(secretName)
    +    }
    +  }
    +
    +  private def runSparkPiAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String] = Array.empty[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PI_MAIN_CLASS,
    +      Seq("Pi is roughly 3"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkPageRankAndVerifyCompletion(
    +      appResource: String = containerLocalSparkDistroExamplesJar,
    +      driverPodChecker: Pod => Unit = doBasicDriverPodCheck,
    +      executorPodChecker: Pod => Unit = doBasicExecutorPodCheck,
    +      appArgs: Array[String],
    +      appLocator: String = appLocator): Unit = {
    +    runSparkApplicationAndVerifyCompletion(
    +      appResource,
    +      SPARK_PAGE_RANK_MAIN_CLASS,
    +      Seq("1 has rank", "2 has rank", "3 has rank", "4 has rank"),
    +      appArgs,
    +      driverPodChecker,
    +      executorPodChecker,
    +      appLocator)
    +  }
    +
    +  private def runSparkApplicationAndVerifyCompletion(
    +      appResource: String,
    +      mainClass: String,
    +      expectedLogOnCompletion: Seq[String],
    +      appArgs: Array[String],
    +      driverPodChecker: Pod => Unit,
    +      executorPodChecker: Pod => Unit,
    +      appLocator: String): Unit = {
    +    val appArguments = SparkAppArguments(
    +      mainAppResource = appResource,
    +      mainClass = mainClass,
    +      appArgs = appArgs)
    +    SparkAppLauncher.launch(appArguments, sparkAppConf, TIMEOUT.value.toSeconds.toInt, sparkHomeDir)
    +
    +    val driverPod = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "driver")
    +      .list()
    +      .getItems
    +      .get(0)
    +    driverPodChecker(driverPod)
    +
    +    val executorPods = kubernetesTestComponents.kubernetesClient
    +      .pods()
    +      .withLabel("spark-app-locator", appLocator)
    +      .withLabel("spark-role", "executor")
    +      .list()
    +      .getItems
    +    executorPods.asScala.foreach { pod =>
    +      executorPodChecker(pod)
    +    }
    +
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      expectedLogOnCompletion.foreach { e =>
    +        assert(kubernetesTestComponents.kubernetesClient
    +          .pods()
    +          .withName(driverPod.getMetadata.getName)
    +          .getLog
    +          .contains(e), "The application did not complete.")
    +      }
    +    }
    +  }
    +
    +  private def doBasicDriverPodCheck(driverPod: Pod): Unit = {
    +    assert(driverPod.getMetadata.getName === driverPodName)
    +    assert(driverPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver")
    +  }
    +
    +  private def doBasicExecutorPodCheck(executorPod: Pod): Unit = {
    +    assert(executorPod.getSpec.getContainers.get(0).getImage === image)
    +    assert(executorPod.getSpec.getContainers.get(0).getName === "executor")
    +  }
    +
    +  private def checkCustomSettings(pod: Pod): Unit = {
    +    assert(pod.getMetadata.getLabels.get("label1") === "label1-value")
    +    assert(pod.getMetadata.getLabels.get("label2") === "label2-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation1") === "annotation1-value")
    +    assert(pod.getMetadata.getAnnotations.get("annotation2") === "annotation2-value")
    +
    +    val container = pod.getSpec.getContainers.get(0)
    +    val envVars = container
    +      .getEnv
    +      .asScala
    +      .map { env =>
    +        (env.getName, env.getValue)
    +      }
    +      .toMap
    +    assert(envVars("ENV1") === "VALUE1")
    +    assert(envVars("ENV2") === "VALUE2")
    +  }
    +
    +  private def deleteDriverPod(): Unit = {
    +    kubernetesTestComponents.kubernetesClient.pods().withName(driverPodName).delete()
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      assert(kubernetesTestComponents.kubernetesClient
    +        .pods()
    +        .withName(driverPodName)
    +        .get() == null)
    +    }
    +  }
    +
    +  private def createTestSecret(secretName: String): Unit = {
    +    kubernetesTestComponents.kubernetesClient.secrets
    +      .createNew()
    +      .editOrNewMetadata()
    +        .withName(secretName)
    +        .endMetadata()
    +      .addToStringData(TEST_SECRET_KEY, TEST_SECRET_VALUE)
    +      .done()
    +  }
    +
    +  private def checkTestSecret(
    +      secretName: String,
    +      pod: Pod,
    +      withInitContainer: Boolean = false): Unit = {
    +    val testSecretVolume = pod.getSpec.getVolumes.asScala.filter { volume =>
    +      volume.getName == s"$secretName-volume"
    +    }
    +    assert(testSecretVolume.size === 1)
    +    assert(testSecretVolume.head.getSecret.getSecretName === secretName)
    +
    +    checkTestSecretInContainer(secretName, pod.getSpec.getContainers.get(0))
    +
    +    if (withInitContainer) {
    +      checkTestSecretInContainer(secretName, pod.getSpec.getInitContainers.get(0))
    +    }
    +  }
    +
    +  private def checkTestSecretInContainer(secretName: String, container: Container): Unit = {
    +    val testSecret = container.getVolumeMounts.asScala.filter { mount =>
    +      mount.getName == s"$secretName-volume"
    +    }
    +    assert(testSecret.size === 1)
    +    assert(testSecret.head.getMountPath === TEST_SECRET_MOUNT_PATH)
    +  }
    +
    +  private def deleteTestSecret(secretName: String): Unit = {
    +    kubernetesTestComponents.kubernetesClient.secrets
    +      .withName(secretName)
    +      .delete()
    +
    +    Eventually.eventually(TIMEOUT, INTERVAL) {
    +      assert(kubernetesTestComponents.kubernetesClient.secrets.withName(secretName).get() == null)
    +    }
    +  }
    +}
    +
    +private[spark] object KubernetesSuite {
    +
    +  val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
    +  val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
    +  val SPARK_PI_MAIN_CLASS: String = "org.apache.spark.examples.SparkPi"
    +  val SPARK_PAGE_RANK_MAIN_CLASS: String = "org.apache.spark.examples.SparkPageRank"
    +
    +  val TEST_SECRET_NAME_PREFIX = "test-secret-"
    +  val TEST_SECRET_KEY = "test-key"
    +  val TEST_SECRET_VALUE = "test-data"
    +  val TEST_SECRET_MOUNT_PATH = "/etc/secrets"
    +
    +  val CONTAINER_LOCAL_FILE_DOWNLOAD_PATH = "/var/spark-data/spark-files"
    --- End diff --
    
    This logic will need to be removed


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I will take a look at the docs too.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    What about the issue of moving the invocation of the tests into run-tests.py? Erik - do you not believe that is a requirement?


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91261 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91261/testReport)** for PR 20697 at commit [`901edb3`](https://github.com/apache/spark/commit/901edb3ba3a566e5c6737d15e197950abce5131c).


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191567638
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.integrationtest.backend
    +
    +import io.fabric8.kubernetes.client.DefaultKubernetesClient
    +
    +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend
    +
    +private[spark] trait IntegrationTestBackend {
    +  def initialize(): Unit
    +  def getKubernetesClient: DefaultKubernetesClient
    +  def cleanUp(): Unit = {}
    +}
    +
    +private[spark] object IntegrationTestBackendFactory {
    +  val DeployModeConfigKey = "spark.kubernetes.test.deployMode"
    --- End diff --
    
    nit: lower case `d` in the var name


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3766/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3675/
    Test FAILed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3714/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3673/
    Test FAILed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #88739 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/88739/testReport)** for PR 20697 at commit [`84a7779`](https://github.com/apache/spark/commit/84a7779f50a0e4d4ca3ef01f5c3e430c681e569c).
     * 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 issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I fixed all the other issues (removal of cloud-based k8s backend, repository clone logic).
    
    @erikerlandson @mccheah @vanzin PTAL.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I still see all the code related to cloning and build Spark source code as part of the PR? (See Matt's old commend about the script that is poorly documented.)


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3575/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/1359/
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191474516
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Logging.scala ---
    @@ -0,0 +1,35 @@
    +/*
    + * 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.integrationtest
    +
    +import org.apache.log4j.{Logger, LogManager, Priority}
    +
    +trait Logging {
    --- End diff --
    
    Resolved in cfb8ee94e11b4871f9b8c7db4774bdb6cb42c40e


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r192452101
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.integrationtest
    +
    +import java.io.File
    +import java.nio.file.{Path, Paths}
    +import java.util.UUID
    +import java.util.regex.Pattern
    +
    +import scala.collection.JavaConverters._
    +
    +import com.google.common.io.PatternFilenameFilter
    +import io.fabric8.kubernetes.api.model.{Container, Pod}
    +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
    +import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
    +import org.scalatest.time.{Minutes, Seconds, Span}
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory}
    +import org.apache.spark.deploy.k8s.integrationtest.config._
    +
    +private[spark] class KubernetesSuite extends SparkFunSuite
    +  with BeforeAndAfterAll with BeforeAndAfter {
    +
    +  import KubernetesSuite._
    +
    +  private var testBackend: IntegrationTestBackend = _
    +  private var sparkHomeDir: Path = _
    +  private var kubernetesTestComponents: KubernetesTestComponents = _
    +  private var sparkAppConf: SparkAppConf = _
    +  private var image: String = _
    +  private var containerLocalSparkDistroExamplesJar: String = _
    +  private var appLocator: String = _
    +  private var driverPodName: String = _
    +
    +  override def beforeAll(): Unit = {
    +    // The scalatest-maven-plugin gives system properties that are referenced but not set null
    +    // values. We need to remove the null-value properties before initializing the test backend.
    +    val nullValueProperties = System.getProperties.asScala
    +      .filter(entry => entry._2.equals("null"))
    +      .map(entry => entry._1.toString)
    +    nullValueProperties.foreach { key =>
    +      System.clearProperty(key)
    +    }
    +
    +    val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir")
    +    require(sparkDirProp != null, "Spark home directory must be provided in system properties.")
    +    sparkHomeDir = Paths.get(sparkDirProp)
    +    require(sparkHomeDir.toFile.isDirectory,
    +      s"No directory found for spark home specified at $sparkHomeDir.")
    +    val imageTag = getTestImageTag
    +    val imageRepo = getTestImageRepo
    +    image = s"$imageRepo/spark:$imageTag"
    +
    +    val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars"))
    +      .toFile
    +      .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0)
    +    containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" +
    +      s"${sparkDistroExamplesJarFile.getName}"
    +    testBackend = IntegrationTestBackendFactory.getTestBackend
    +    testBackend.initialize()
    +    kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    testBackend.cleanUp()
    +  }
    +
    +  before {
    +    appLocator = UUID.randomUUID().toString.replaceAll("-", "")
    +    driverPodName = "spark-test-app-" + UUID.randomUUID().toString.replaceAll("-", "")
    +    sparkAppConf = kubernetesTestComponents.newSparkAppConf()
    +      .set("spark.kubernetes.container.image", image)
    +      .set("spark.kubernetes.driver.pod.name", driverPodName)
    +      .set("spark.kubernetes.driver.label.spark-app-locator", appLocator)
    +      .set("spark.kubernetes.executor.label.spark-app-locator", appLocator)
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.createNamespace()
    +    }
    +  }
    +
    +  after {
    +    if (!kubernetesTestComponents.hasUserSpecifiedNamespace) {
    +      kubernetesTestComponents.deleteNamespace()
    +    }
    +    deleteDriverPod()
    +  }
    +
    +  test("Run SparkPi with no resources") {
    --- End diff --
    
    Looks like they got accidentally removed, re-adding.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    I rarely use it directly. You can hack it to just run what you want locally, for example, if you can't figure out another way.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-spark-integration/3628/



---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Correct. I will change the K8s integration tests to do exactly what you describe above, after this is merged. @vanzin do you want to comment? One of the two of you will need to do the merge, of course.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    **[Test build #91172 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91172/testReport)** for PR 20697 at commit [`13721f6`](https://github.com/apache/spark/commit/13721f69a21c91c0b42a5471b50b8f53e4c7808f).


---

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


[GitHub] spark pull request #20697: [SPARK-23010][k8s] Initial checkin of k8s integra...

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

    https://github.com/apache/spark/pull/20697#discussion_r191855713
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala ---
    @@ -0,0 +1,43 @@
    +/*
    + * 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.integrationtest.backend
    +
    +import io.fabric8.kubernetes.client.DefaultKubernetesClient
    +
    +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend
    +
    +private[spark] trait IntegrationTestBackend {
    +  def initialize(): Unit
    +  def getKubernetesClient: DefaultKubernetesClient
    +  def cleanUp(): Unit = {}
    +}
    +
    +private[spark] object IntegrationTestBackendFactory {
    +  val DeployModeConfigKey = "spark.kubernetes.test.deployMode"
    --- 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 #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3734/
    Test PASSed.


---

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


[GitHub] spark issue #20697: Initial checkin of k8s integration tests.

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

    https://github.com/apache/spark/pull/20697
  
    I should note that the integration test success that came from SparkQA is **not** using the copy of the integration test code mentioned in this PR. It is using the original copy of the code in https://github.com/apache-spark-on-k8s/spark-integration. Once this is merged, I will create additional Jenkins jobs on the AMPLab infrastructure that exercises this new copy of the integration test. Once those are working, I will remove the Jenkins jobs that use the apache-spark-on-k8s/spark-integration repo.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

    https://github.com/apache/spark/pull/20697
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3597/
    Test PASSed.


---

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


[GitHub] spark issue #20697: [SPARK-23010][k8s] Initial checkin of k8s integration te...

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

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


---

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