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

2018-06-08 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



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

2018-06-08 Thread ssuchter
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) {

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

2018-06-08 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-06-08 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-06-08 Thread ssuchter
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) {

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

2018-06-08 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-06-08 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-06-08 Thread ifilonenko
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) 

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

2018-06-07 Thread ssuchter
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) {

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

2018-06-01 Thread ssuchter
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) {

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

2018-06-01 Thread ssuchter
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) {

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

2018-06-01 Thread ssuchter
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) {

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

2018-06-01 Thread ssuchter
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) {

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

2018-05-31 Thread ifilonenko
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) 

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

2018-05-31 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/20697#discussion_r192270901
  
--- 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 --

Yup that makes sense. Thanks!


---

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



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

2018-05-30 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-30 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-29 Thread ssuchter
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
+1.1.1
+5.0.2
+1.3.0
+1.4.0
+
+18.0
+1.3.9
+3.0.0
+1.2.17
+2.11.8
+2.11
+3.2.2
+2.2.6
+1.0
+1.7.24
+kubernetes-integration-tests
+
${project.build.directory}/spark-dist-unpacked
+N/A
+
${project.build.directory}/imageTag.txt
+
minikube
+
docker.io/kubespark
+
+  
+  jar
+  Spark Project Kubernetes Integration Tests
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  commons-logging
+  commons-logging
+  ${commons-logging.version}
+
+
+  com.google.code.findbugs
+  jsr305
+  ${jsr305.version}
+
+
+  com.google.guava
+  guava
+  test
+  
+  ${guava.version}
+
+
+  com.spotify
+  docker-client
+  ${docker-client.version}
+  test
+
+
+  io.fabric8
+  kubernetes-client
+  ${kubernetes-client.version}
+
+
+  log4j
+  log4j
+  ${log4j.version}
+
+
+  org.apache.commons
+  commons-lang3
+  ${commons-lang3.version}
+
+
+  org.scala-lang
+  scala-library
+  ${scala.version}
+
+
+  org.scalatest
+  scalatest_${scala.binary.version}
+  test
+
+
+  org.slf4j
+  slf4j-log4j12
+  ${slf4j-log4j12.version}
+  test
+
+  
+
+  
+
+  
+net.alchim31.maven
--- End diff --

Resolved in 5a8fd7ff40


---

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



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

2018-05-29 Thread ssuchter
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
+1.1.1
+5.0.2
+1.3.0
+1.4.0
+
+18.0
+1.3.9
+3.0.0
+1.2.17
+2.11.8
+2.11
+3.2.2
+2.2.6
+1.0
+1.7.24
+kubernetes-integration-tests
+
${project.build.directory}/spark-dist-unpacked
+N/A
+
${project.build.directory}/imageTag.txt
+
minikube
+
docker.io/kubespark
+
+  
+  jar
+  Spark Project Kubernetes Integration Tests
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  commons-logging
+  commons-logging
+  ${commons-logging.version}
+
+
+  com.google.code.findbugs
+  jsr305
+  ${jsr305.version}
+
+
+  com.google.guava
+  guava
+  test
+  
+  ${guava.version}
+
+
+  com.spotify
+  docker-client
+  ${docker-client.version}
+  test
+
+
+  io.fabric8
+  kubernetes-client
+  ${kubernetes-client.version}
+
+
+  log4j
+  log4j
+  ${log4j.version}
--- End diff --

Resolved in 901edb3ba3, f68cdba77b, dd280327ea


---

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



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

2018-05-29 Thread ssuchter
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
--- End diff --

Resolved in 901edb3ba3, f68cdba77b, dd280327ea


---

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



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

2018-05-29 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/20697#discussion_r191034013
  
--- 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 --

Is log4j configured for us now that we're embedded in the project? Or do we 
still need a separate properties file for this?


---

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



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

2018-05-25 Thread mccheah
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
+1.1.1
+5.0.2
+1.3.0
+1.4.0
+
+18.0
+1.3.9
+3.0.0
+1.2.17
+2.11.8
+2.11
+3.2.2
+2.2.6
+1.0
+1.7.24
+kubernetes-integration-tests
+
${project.build.directory}/spark-dist-unpacked
+N/A
+
${project.build.directory}/imageTag.txt
+
minikube
+
docker.io/kubespark
+
+  
+  jar
+  Spark Project Kubernetes Integration Tests
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  commons-logging
+  commons-logging
+  ${commons-logging.version}
+
+
+  com.google.code.findbugs
+  jsr305
+  ${jsr305.version}
+
+
+  com.google.guava
+  guava
+  test
+  
+  ${guava.version}
+
+
+  com.spotify
+  docker-client
+  ${docker-client.version}
+  test
+
+
+  io.fabric8
+  kubernetes-client
+  ${kubernetes-client.version}
+
+
+  log4j
+  log4j
+  ${log4j.version}
+
+
+  org.apache.commons
+  commons-lang3
+  ${commons-lang3.version}
+
+
+  org.scala-lang
+  scala-library
+  ${scala.version}
+
+
+  org.scalatest
+  scalatest_${scala.binary.version}
+  test
+
+
+  org.slf4j
+  slf4j-log4j12
+  ${slf4j-log4j12.version}
+  test
+
+  
+
+  
+
+  
+net.alchim31.maven
--- 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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread mccheah
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
--- 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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread mccheah
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 @@
+
+
+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;>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.4.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes-integration-tests_2.11
+  spark-kubernetes-integration-tests
+  
+3.3.9
+3.5
+1.1.1
+5.0.2
+1.3.0
+1.4.0
+
+18.0
+1.3.9
+3.0.0
+1.2.17
+2.11.8
+2.11
+3.2.2
+2.2.6
+1.0
+1.7.24
+kubernetes-integration-tests
+
${project.build.directory}/spark-dist-unpacked
+N/A
+
${project.build.directory}/imageTag.txt
+
minikube
+
docker.io/kubespark
+
+  
+  jar
+  Spark Project Kubernetes Integration Tests
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  commons-logging
+  commons-logging
+  ${commons-logging.version}
+
+
+  com.google.code.findbugs
+  jsr305
+  ${jsr305.version}
+
+
+  com.google.guava
+  guava
+  test
+  
+  ${guava.version}
+
+
+  com.spotify
+  docker-client
+  ${docker-client.version}
+  test
+
+
+  io.fabric8
+  kubernetes-client
+  ${kubernetes-client.version}
+
+
+  log4j
+  log4j
+  ${log4j.version}
--- End diff --

...and all these versions should be provided by the parent pom too.


---

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



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

2018-05-25 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread ssuchter
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) {
+  

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

2018-05-25 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-05-25 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread ssuchter
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



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

2018-03-06 Thread mccheah
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-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org