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 2022/12/07 00:31:14 UTC

[GitHub] [spark] tedyu commented on pull request #38943: [SPARK-41410][K8S] Support PVC-oriented executor pod allocation

tedyu commented on PR #38943:
URL: https://github.com/apache/spark/pull/38943#issuecomment-1340205543

   I think the `PVC_COUNTER` should only be decremented when the pod deletion happens (in response to error).
   @dongjoon-hyun 
   What do you think of the following change ?
   ```
   diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
   index 4188a9038a..63fc29ea80 100644
   --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
   +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
   @@ -433,6 +433,7 @@ class ExecutorPodsAllocator(
            podWithAttachedContainer, resolvedExecutorSpec.executorKubernetesResources, reusablePVCs)
          val createdExecutorPod =
            kubernetesClient.pods().inNamespace(namespace).resource(podWithAttachedContainer).create()
   +      var success = 0
          try {
            addOwnerReference(createdExecutorPod, resources)
            resources
   @@ -445,6 +446,7 @@ class ExecutorPodsAllocator(
                logInfo(s"Trying to create PersistentVolumeClaim ${pvc.getMetadata.getName} with " +
                  s"StorageClass ${pvc.getSpec.getStorageClassName}")
                kubernetesClient.persistentVolumeClaims().inNamespace(namespace).resource(pvc).create()
   +            success += 1
                PVC_COUNTER.incrementAndGet()
              }
            newlyCreatedExecutors(newExecutorId) = (resourceProfileId, clock.getTimeMillis())
   @@ -455,8 +457,12 @@ class ExecutorPodsAllocator(
                .inNamespace(namespace)
                .resource(createdExecutorPod)
                .delete()
   -          PVC_COUNTER.decrementAndGet()
   +          if (success == 1) {
   +            success += 1
   +          }
              throw e
   +      } finally {
   +          if (success == 2) PVC_COUNTER.decrementAndGet()
          }
        }
      }
   ```
   If pvc isn't created successfully, it seems the counter shouldn't be decremented.


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