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/12/17 02:53:40 UTC

[GitHub] [spark] Yikun commented on a change in pull request #34599: [SPARK-37331][K8S] Add the ability to create resources before driverPod creating

Yikun commented on a change in pull request #34599:
URL: https://github.com/apache/spark/pull/34599#discussion_r771069599



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
##########
@@ -133,15 +133,39 @@ private[spark] class Client(
       .build()
     val driverPodName = resolvedDriverPod.getMetadata.getName
 
+    // setup resources before pod creation
+    val preKubernetesResources = resolvedDriverSpec.driverPreKubernetesResources
+    try {
+      kubernetesClient.resourceList(preKubernetesResources: _*).createOrReplace()
+    } catch {
+      case NonFatal(e) =>
+        kubernetesClient.resourceList(preKubernetesResources: _*).delete()
+        throw e
+    }
+
     var watch: Watch = null
     var createdDriverPod: Pod = null
     try {
       createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod)
     } catch {
       case NonFatal(e) =>
+        kubernetesClient.resourceList(preKubernetesResources: _*).delete()
         logError("Please check \"kubectl auth can-i create pod\" first. It should be yes.")
         throw e
     }
+
+    // Refresh all pre-resources' owner references
+    try {
+      addOwnerReference(createdDriverPod, preKubernetesResources)
+      kubernetesClient.resourceList(preKubernetesResources: _*).createOrReplace()
+    } catch {
+      case NonFatal(e) =>
+        kubernetesClient.pods().delete(createdDriverPod)
+        kubernetesClient.resourceList(preKubernetesResources: _*).delete()

Review comment:
       > What happens when the previous line deletes some of the resources of preKubernetesResources? For the non-existing resource, delete() API is okay?
   
   Yes, it's okay, `false` return in here instead of exception rasing. We could find answer in kubernetes-client [delete implementation](https://github.com/fabric8io/kubernetes-client/blob/d379b0b2af2541a4378f6efcc7e535a282355c20/kubernetes-client/src/main/java/io/fabric8/kubernetes/client/dsl/base/BaseOperation.java#L440-L462) and [resource list delete test](https://github.com/fabric8io/kubernetes-client/blob/f45ed4f06e98c6840b27f1e5a44d3ba4d37b682d/kubernetes-tests/src/test/java/io/fabric8/kubernetes/client/mock/ResourceListTest.java#L131).
   
   I also write a simple test to delete non-existing crd in real env, it passed as expected.
   ```scala
     test("Deleting nonExisting crds") {
       val crd = new CustomResourceDefinitionBuilder()
         .withNewMetadata()
         .withName("nonExisting1")
         .endMetadata()
         .build()
       val crd2 = new CustomResourceDefinitionBuilder()
         .withNewMetadata()
         .withName("nonExisting")
         .endMetadata()
         .build()
       val crds = Seq(crd, crd2)
       val client = new DefaultKubernetesClient("https://127.0.0.1:52878")
       assert(client.inNamespace("default").resourceList(crd).delete() === false)
       assert(client.inNamespace("default").resourceList(crds: _*).delete() == false)
     }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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