You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/14 06:38:11 UTC

[GitHub] [spark] attilapiros opened a new pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

attilapiros opened a new pull request #31829:
URL: https://github.com/apache/spark/pull/31829


   
   ### What changes were proposed in this pull request?
   
   This PR upgrades Kubernetes and Minikube version for integration tests and removes/updates the old code for this new version.
   
   Details of this changes: 
   
   - As [discussed in the mailing list](http://apache-spark-developers-list.1001551.n3.nabble.com/minikube-and-kubernetes-cluster-versions-for-integration-testing-td30856.html): updating Minikube version from v0.34.1 to v1.7.3 and kubernetes version from v1.15.12 to v1.17.3.
   - making Minikube version checked and fail with an explanation when the test is started with on a version <  v1.7.3.
   - removing minikube status checking code related to old Minikube versions 
   - in the Minikube backend using fabric8's `Config.autoConfigure()` method to configure the kubernetes client to use the `minikube` k8s context (like it was in [one of the Minikube's example](https://github.com/fabric8io/kubernetes-client/blob/master/kubernetes-examples/src/main/java/io/fabric8/kubernetes/examples/kubectl/equivalents/ConfigUseContext.java#L36))
   
   ### Why are the changes needed?
   
   With the current suggestion one can run into several problems without noticing the Minikube/kubernetes version is the problem.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   It was tested on Mac with [this script](https://gist.github.com/attilapiros/cd58a16bdde833c80c5803c337fffa94#file-check_minikube_versions-zsh) which installs each Minikube versions from v1.7.2 (including this version to test the negative case of the version check) and runs the integration tests.
   
   It was started with:
   ```
   ./check_minikube_versions.zsh > test_log 2>&1
   ```
   
   And there was only one build failure the rest was successful:
   
   ```
   $ grep "BUILD SUCCESS" test_log | wc -l
         26
   $ grep "BUILD FAILURE" test_log | wc -l
          1
   ```
   
   It was for Minikube v1.7.2  and the log is:
   
   ```
   KubernetesSuite:
   *** RUN ABORTED ***
     java.lang.AssertionError: assertion failed: Unsupported Minikube version is detected: minikube version: v1.7.2.For integration testing Minikube version 1.7.3 or greater is expected.
     at scala.Predef$.assert(Predef.scala:223)
     at org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube$.getKubernetesClient(Minikube.scala:52)
     at org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend$.initialize(MinikubeTestBackend.scala:33)
     at org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.beforeAll(KubernetesSuite.scala:163)
     at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212)
     at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
     at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
     at org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.org$scalatest$BeforeAndAfter$$super$run(KubernetesSuite.scala:43)
     at org.scalatest.BeforeAndAfter.run(BeforeAndAfter.scala:273)
     at org.scalatest.BeforeAndAfter.run$(BeforeAndAfter.scala:271)
     ...
   ```
   
   Moreover I made a test with having multiple k8s clusters contexts, too.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-836953989


   Thanks @dongjoon-hyun. 
   
   Tested locally:
   
   ```
   [INFO] --- scalatest-maven-plugin:2.0.0:test (integration-test) @ spark-kubernetes-integration-tests_2.12 ---
   Discovery starting.
   Discovery completed in 446 milliseconds.
   Run starting. Expected test count is: 25
   KubernetesSuite:
   - Run SparkPi with no resources
   - Run SparkPi with a very long application name.
   - Use SparkLauncher.NO_RESOURCE
   - Run SparkPi with a master URL without a scheme.
   - Run SparkPi with an argument.
   - Run SparkPi with custom labels, annotations, and environment variables.
   - All pods have the same service account by default
   - Run extraJVMOptions check on driver
   - Run SparkRemoteFileTest using a remote data file
   - Verify logging configuration is picked from the provided SPARK_CONF_DIR/log4j.properties
   - Run SparkPi with env and mount secrets.
   - Run PySpark on simple pi.py example
   - Run PySpark to test a pyfiles example
   - Run PySpark with memory customization
   - Run in client mode.
   - Start pod creation from template
   - Launcher client dependencies
   - SPARK-33615: Launcher client archives
   - SPARK-33748: Launcher python client respecting PYSPARK_PYTHON
   - SPARK-33748: Launcher python client respecting spark.pyspark.python and spark.pyspark.driver.python
   - Launcher python client dependencies using a zip file
   - Test basic decommissioning
   - Test basic decommissioning with shuffle cleanup
   - Test decommissioning with dynamic allocation & shuffle cleanups
   - Test decommissioning timeouts
   Run completed in 16 minutes, 51 seconds.
   Total number of tests run: 25
   ```  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-798850315


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40621/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628917445



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "

Review comment:
       Fine for me.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821767457


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822189802


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137566/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821767500


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137499/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821771986


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42073/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628835311



##########
File path: resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
##########
@@ -176,4 +180,4 @@ properties+=(
   -Dlog4j.logger.org.apache.spark=DEBUG
 )
 
-$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}
+$TEST_ROOT_DIR/build/mvn install -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests $BUILD_DEPENDENCIES_MVN_FLAG -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}

Review comment:
       In addition, this is inconsistent from README.md line 128.
   - https://github.com/apache/spark/pull/31829/files#diff-280a1f3f6bebe922254b2a1f2f952929f02e582c60169c659f1dff792f633caaR128




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] fbalicchia commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
fbalicchia commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-814414886


   Using minikube `v1.18.1` default driver is Docker and at the moment of writing masterUrl not necessary bind on port `8443` https://github.com/apache/spark/blob/master/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala#L69.
   Instead using default config or using factory with `autoConfigure("minikube")` library help us


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821790383






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628835797



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "

Review comment:
       Most of changes are only related to these spacing changes. I'm wondering if these changes are inevitable for the purpose of this PR.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822175175


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821771986


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42073/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594501571



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       I have tested this case and when after minikube is started and the context is switched to another one even that makes the test fail. 
   
   So the best probably is mentioning that fact too:
   ```
   - the current kubernetes context must be minikube's default context (called 'minikube'). This can be selected by `minikube kubectl -- config use-context minikube`. This is only needed when after minikube is started another kubernetes context is selected.
   ```
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628835907



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala
##########
@@ -565,6 +565,7 @@ class KubernetesSuite extends SparkFunSuite
 
 private[spark] object KubernetesSuite {
   val k8sTestTag = Tag("k8s")
+  val pvTestTag = Tag("persistentVolume")

Review comment:
       Is this new tag documented or mentioned somewhere in this PR?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821787019


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815638835


   **[Test build #137071 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137071/testReport)** for PR 31829 at commit [`de32835`](https://github.com/apache/spark/commit/de32835821b9d349faba6ff237a43c077b06534c).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628898709



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "

Review comment:
       Then, if you don't mind, shall you split that refactoring contribution into another PR? Each PR had better have one theme and minimal.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-798851584






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] fbalicchia edited a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
fbalicchia edited a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-814304727


   Hi, I'm experiencing in the same issue and and to resolve the issue I've applied the same changes. Could you please merge it ?
   Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821791138






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] fbalicchia commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
fbalicchia commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-814304727


   Hi, I'm experiencing in the same issue and and to resolve the issue I've applied the same changes. Could you please merge it ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628834835



##########
File path: resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
##########
@@ -176,4 +180,4 @@ properties+=(
   -Dlog4j.logger.org.apache.spark=DEBUG
 )
 
-$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}
+$TEST_ROOT_DIR/build/mvn install -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests $BUILD_DEPENDENCIES_MVN_FLAG -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}

Review comment:
       Well, I know that we need `install` sometimes, but for this one, I'm a little negative for converting `integration-test` to `install` because it pollutes the local repo while testing and reviewing someone-else's PR. It's really painful for committers to clean up the local repo everytime to avoid any side-effects for the other PRs. Is this really inevitable?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815638835


   **[Test build #137071 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137071/testReport)** for PR 31829 at commit [`de32835`](https://github.com/apache/spark/commit/de32835821b9d349faba6ff237a43c077b06534c).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821767500


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137499/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821785076


   rebased on master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-798850804


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40621/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] rvesse commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
rvesse commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594433527



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       I'm sure it's probably is the default (though their community will be best placed to confirm).  The scenario I was thinking of was more the one where users have renamed their contexts to their own preference.  I have access to a lot of R&D clusters in my $dayjob and at times I've had as many as 10+ different contexts in my `KUBECONFIG` so I would rename contexts appropriately as some tools (GKE I'm looking at you) produce rather user unfriendly names by default.  Not sure I've ever renamed the `minikube` context itself but it's a thing that users can and might do.
   
   TL;DR If there's a naming assumption present in the code, particularly where it pertains to user managed configuration files outside of Spark's control, it should be documented as such.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815665399


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41649/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] github-actions[bot] commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821785067


   **[Test build #758035203](https://github.com/attilapiros/spark/actions/runs/758035203)** for PR 31829 at commit [`855273a`](https://github.com/attilapiros/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821766064


   **[Test build #137499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137499/testReport)** for PR 31829 at commit [`1f594aa`](https://github.com/apache/spark/commit/1f594aa40061da96545d29d28ca2b2a410655c34).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594422864



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       I think that is the default context for minikube. Although I have not found this in documentation so I asked this in the Minikube user's list:
   
   ```
   Hi!
   
   As I see the default context for the started cluster is `minikube` but I have not found this in documentation.
   - Am I right? 
   - Can I rely on this?
   - Could you please show me the documentation about this (or the relevant code part)?
   
   Thanks in advance,
   Attila
   ```
   
   I hope we get an answer soon. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-830012827


   @dongjoon-hyun actually I am pretty sure this helps (even with the skipping of the PVTests which when @shaneknapp sets the Minikube driver to Docker will fix the Kubernetes integration tests).
   
   So could you please review this as it is?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-798851584






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822166832


   **[Test build #137566 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137566/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815648141


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137071/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r609529325



##########
File path: resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
##########
@@ -176,4 +180,4 @@ properties+=(
   -Dlog4j.logger.org.apache.spark=DEBUG
 )
 
-$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}
+$TEST_ROOT_DIR/build/mvn install -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests $BUILD_DEPENDENCIES_MVN_FLAG -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}

Review comment:
       The `install` phase is after `integration-test`, see default lifecycle at the [maven doc](https://maven.apache.org/guides/introduction/introduction-to-the-lifecycle.html):
   
   
   Phase | Description
   -- | --
   ... | ...
   package | take the compiled code and package it in its distributable format, such as a JAR.
   pre-integration-test | perform actions required before integration tests are executed. This may involve things such as setting up the required environment.
   **integration-test** | process and deploy the package if necessary into an environment where integration tests can be run.
   post-integration-test | perform actions required after integration tests have been executed. This may including cleaning up the environment.
   verify | run any checks to verify the package is valid and meets quality criteria.
   **install** | install the package into the local repository, for use as a dependency in other projects locally.
   ...|...
   
   
   With `install` the local repo will be updated with the fresh jars and in case of `--skip-building-dependencies` the build will reuse those.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628837167



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala
##########
@@ -565,6 +565,7 @@ class KubernetesSuite extends SparkFunSuite
 
 private[spark] object KubernetesSuite {
   val k8sTestTag = Tag("k8s")
+  val pvTestTag = Tag("persistentVolume")

Review comment:
       This would be a temporary change to enable Jenkins to use Docker as minikube's driver.
   This way Kubernetes integration test would be successful.
   
   More about it:
   https://issues.apache.org/jira/browse/SPARK-34738?focusedCommentId=17324165&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17324165




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815648141






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822340493


   **[Test build #137602 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137602/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822353999


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137602/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821766064


   **[Test build #137499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137499/testReport)** for PR 31829 at commit [`1f594aa`](https://github.com/apache/spark/commit/1f594aa40061da96545d29d28ca2b2a410655c34).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821770063


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42073/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-835692511


   I left a few comments.
   - https://github.com/apache/spark/pull/31829#discussion_r628834835
   - https://github.com/apache/spark/pull/31829#discussion_r628835311
   - https://github.com/apache/spark/pull/31829#discussion_r628835797
   - https://github.com/apache/spark/pull/31829#discussion_r628835907


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822353568


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821785178


   **[Test build #137502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137502/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros closed pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros closed pull request #31829:
URL: https://github.com/apache/spark/pull/31829


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815675133


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41649/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628853939



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "

Review comment:
       I can revert them but I think this is more clean than polluting each usage with the extra space. In addition we can use the `split` method (and getting the second part) instead of the `replace_first` which express our intention a bit better.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822340493


   **[Test build #137602 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137602/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822353999


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137602/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628853451



##########
File path: resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh
##########
@@ -176,4 +180,4 @@ properties+=(
   -Dlog4j.logger.org.apache.spark=DEBUG
 )
 
-$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}
+$TEST_ROOT_DIR/build/mvn install -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests $BUILD_DEPENDENCIES_MVN_FLAG -Pscala-$SCALA_VERSION -P$HADOOP_PROFILE -Pkubernetes -Pkubernetes-integration-tests ${properties[@]}

Review comment:
       I would say this is the orherway around, assuming in both PRs we stick to `install`.
   So when we call `install` in the first PR we could be sure the fresh jars will be used (even when the artifact we build referencing something outside of the maven reactor) and when we move to the next PR because of `install` we have again the fresh jars.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628917492



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala
##########
@@ -565,6 +565,7 @@ class KubernetesSuite extends SparkFunSuite
 
 private[spark] object KubernetesSuite {
   val k8sTestTag = Tag("k8s")
+  val pvTestTag = Tag("persistentVolume")

Review comment:
       I got it :)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815647857


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821785178


   **[Test build #137502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137502/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r628898201



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala
##########
@@ -565,6 +565,7 @@ class KubernetesSuite extends SparkFunSuite
 
 private[spark] object KubernetesSuite {
   val k8sTestTag = Tag("k8s")
+  val pvTestTag = Tag("persistentVolume")

Review comment:
       Yes, that's exactly what I mean. We had better mention this workaround and limitation (ignoring PVC tests) of this PR in the description for the reviewers.
   > This would be a temporary change to enable Jenkins to use Docker as minikube's driver.
   This way Kubernetes integration test would be successful.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822189802


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137566/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-822166832


   **[Test build #137566 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137566/testReport)** for PR 31829 at commit [`855273a`](https://github.com/apache/spark/commit/855273a67e7e6a62130c707439d15ac25d4cb1f2).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-814397406


   @fbalicchia Could you please share with us the error you run into (the one on which this PR helped on)?
   By the way I cannot merge my own PR without an expert's approval (expert of the area where the PR belongs to). But your input will help the case.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-821791138






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-815668763


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/41649/
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594422864



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       I think that is the default context for minikube. Although I have not found this in documentation so I asked this in the Minikube user's list:
   
   ```
   Hi!
   
   As I see the default context for the started cluster is `minikube` but I have not found this in documentation.
   - Am I right? 
   - Can I really on this?
   - Could you please show me the documentation about this (or the relevant code part)?
   
   Thanks in advance,
   Attila
   ```
   
   I hope we can get an answer soon. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] dongjoon-hyun commented on pull request #31829: [SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #31829:
URL: https://github.com/apache/spark/pull/31829#issuecomment-835685659


   Oh, I missed your last comment here. Very sorry, @attilapiros .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] attilapiros commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594422864



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       I think that is the default context for minikube. Although I have not found this in documentation so I asked this in the Minikube user's list:
   
   ```
   Hi!
   
   As I see the default context for the started cluster is `minikube` but I have not found this in documentation.
   - Am I right? 
   - Can I really on this?
   - Could you please show me the documentation about this (or the relevant code part)?
   
   Thanks in advance,
   Attila
   ```
   
   I hope we get an answer soon. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] rvesse commented on a change in pull request #31829: [WIP][SPARK-34736][K8S][TESTS] Kubernetes and Minikube version upgrade for integration tests

Posted by GitBox <gi...@apache.org>.
rvesse commented on a change in pull request #31829:
URL: https://github.com/apache/spark/pull/31829#discussion_r594240705



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala
##########
@@ -16,100 +16,73 @@
  */
 package org.apache.spark.deploy.k8s.integrationtest.backend.minikube
 
-import java.nio.file.{Files, Paths}
-
-import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+import io.fabric8.kubernetes.client.Config
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
 
 import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils
 import org.apache.spark.internal.Logging
 
 // TODO support windows
 private[spark] object Minikube extends Logging {
   private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
-  private val HOST_PREFIX = "host:"
-  private val KUBELET_PREFIX = "kubelet:"
-  private val APISERVER_PREFIX = "apiserver:"
-  private val KUBECTL_PREFIX = "kubectl:"
-  private val KUBECONFIG_PREFIX = "kubeconfig:"
+  private val VERSION_PREFIX = "minikube version: "
+  private val HOST_PREFIX = "host: "
+  private val KUBELET_PREFIX = "kubelet: "
+  private val APISERVER_PREFIX = "apiserver: "
+  private val KUBECTL_PREFIX = "kubectl: "
+  private val KUBECONFIG_PREFIX = "kubeconfig: "
   private val MINIKUBE_VM_PREFIX = "minikubeVM: "
   private val MINIKUBE_PREFIX = "minikube: "
-  private val MINIKUBE_PATH = ".minikube"
-
-  def logVersion(): Unit = {
-    logInfo(executeMinikube("version").mkString("\n"))
-  }
 
-  def getMinikubeIp: String = {
-    val outputs = executeMinikube("ip")
-      .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
-    assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
-    outputs.head
-  }
+  lazy val minikubeVersionString =
+    executeMinikube("version").find(_.contains(VERSION_PREFIX)).get
 
-  def getMinikubeStatus: MinikubeStatus.Value = {
-    val statusString = executeMinikube("status")
-    logInfo(s"Minikube status command output:\n$statusString")
-    // up to minikube version v0.30.0 use this to check for minikube status
-    val oldMinikube = statusString
-      .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX))
-
-    if (oldMinikube.isEmpty) {
-      getIfNewMinikubeStatus(statusString)
-    } else {
-      val finalStatusString = oldMinikube
-        .head
-        .replaceFirst(MINIKUBE_VM_PREFIX, "")
-        .replaceFirst(MINIKUBE_PREFIX, "")
-      MinikubeStatus.unapply(finalStatusString)
-        .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
-    }
-  }
+  def logVersion(): Unit =
+    logInfo(minikubeVersionString)
 
   def getKubernetesClient: DefaultKubernetesClient = {
-    val kubernetesMaster = s"https://${getMinikubeIp}:8443"
-    val userHome = System.getProperty("user.home")
-    val minikubeBasePath = Paths.get(userHome, MINIKUBE_PATH).toString
-    val profileDir = if (Files.exists(Paths.get(minikubeBasePath, "apiserver.crt"))) {
-      // For Minikube <1.9
-      ""
-    } else {
-      // For Minikube >=1.9
-      Paths.get("profiles", executeMinikube("profile")(0)).toString
+    // only the three-part version number is matched (the optional suffix like "-beta.0" is dropped)
+    val versionArrayOpt = "\\d+\\.\\d+\\.\\d+".r
+      .findFirstIn(minikubeVersionString.split(VERSION_PREFIX)(1))
+      .map(_.split('.').map(_.toInt))
+
+    versionArrayOpt match {
+      case Some(Array(x, y, z)) =>
+        if (Ordering.Tuple3[Int, Int, Int].lt((x, y, z), (1, 7, 3))) {
+          assert(false, s"Unsupported Minikube version is detected: $minikubeVersionString." +
+            "For integration testing Minikube version 1.7.3 or greater is expected.")
+        }
+      case _ =>
+        assert(false, s"Unexpected version format detected in `$minikubeVersionString`." +
+          "For minikube version a three-part version number is expected (the optional " +
+          "non-numeric suffix is intentionally dropped)")
     }
-    val apiServerCertPath = Paths.get(minikubeBasePath, profileDir, "apiserver.crt")
-    val apiServerKeyPath = Paths.get(minikubeBasePath, profileDir, "apiserver.key")
-    val kubernetesConf = new ConfigBuilder()
-      .withApiVersion("v1")
-      .withMasterUrl(kubernetesMaster)
-      .withCaCertFile(
-        Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath)
-      .withClientCertFile(apiServerCertPath.toFile.getAbsolutePath)
-      .withClientKeyFile(apiServerKeyPath.toFile.getAbsolutePath)
-      .build()
-    new DefaultKubernetesClient(kubernetesConf)
+
+    new DefaultKubernetesClient(Config.autoConfigure("minikube"))

Review comment:
       If we're now enforcing that the name of the context must be `minikube` this should be included in the apache/spark-website#321 PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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