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 2020/09/25 23:04:44 UTC

[GitHub] [spark] nssalian opened a new pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

nssalian opened a new pull request #29877:
URL: https://github.com/apache/spark/pull/29877


   ### What changes were proposed in this pull request? 
   
   Similar patch to #24748 but applied to the branch-2.4.
   Backporting the fix to releases 2.4.x.
   Please let me know if I missed some step; I haven't contributed to spark in a long time.
   


----------------------------------------------------------------
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] nssalian commented on a change in pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
##########
@@ -61,10 +61,9 @@ private[spark] object Config extends Logging {
       .stringConf
       .createOptional
 
-  val KUBERNETES_AUTH_DRIVER_CONF_PREFIX =
-      "spark.kubernetes.authenticate.driver"
-  val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
-      "spark.kubernetes.authenticate.driver.mounted"
+  val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver"
+  val KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX = "spark.kubernetes.authenticate.executor"
+  val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted"

Review comment:
       The linters didn't fail so I think we are ok.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   @dongjoon-hyun please let me know if you have additional comments. Tests seem to have passed.


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala
##########
@@ -16,6 +16,10 @@
  */
 package org.apache.spark.deploy.k8s
 
+import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder,
+  ContainerStateRunning, ContainerStateTerminated,
+  ContainerStateWaiting, ContainerStatus, Pod, PodBuilder}

Review comment:
       Please use a single line like the original branch, @nssalian .




----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Checking again for this PR:  @dongjoon-hyun @erikerlandson.


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   **[Test build #129467 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129467/testReport)** for PR 29877 at commit [`dafd0ac`](https://github.com/apache/spark/commit/dafd0ace9ffd942e89d04dd90e7564bd717983ba).


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   **[Test build #129525 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129525/testReport)** for PR 29877 at commit [`489fa25`](https://github.com/apache/spark/commit/489fa25bf0cb8d23802838178fe38b894b694b6c).
    * 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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   > Can we have java doc here?
   
   There's no javadoc in the original PR (https://github.com/apache/spark/pull/24748). This is simply a backport. Any improvements can follow in a different 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] dongjoon-hyun commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   ok to test


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   ping:  @dongjoon-hyun @erikerlandson, please review when you get a chance.


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


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


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala
##########
@@ -70,15 +71,8 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube
 
   override def configurePod(pod: SparkPod): SparkPod = {
     if (!shouldMountSecret) {
-      pod.copy(
-        pod = driverServiceAccount.map { account =>
-          new PodBuilder(pod.pod)
-            .editOrNewSpec()
-              .withServiceAccount(account)
-              .withServiceAccountName(account)
-              .endSpec()
-            .build()
-        }.getOrElse(pod.pod))
+      pod.copy(pod = buildPodWithServiceAccount(driverServiceAccount, pod)
+        .getOrElse(pod.pod))

Review comment:
       Why do we need to split this line? The original code is one-liner in `master` branch.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.k8s.features
+
+import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod}
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME
+import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount
+
+private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_])
+  extends KubernetesFeatureConfigStep {
+  private lazy val driverServiceAccount =
+    kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME)
+
+  override def configurePod(pod: SparkPod): SparkPod = {
+    pod.copy(
+      // if not setup by the pod template fallback to the driver's sa,
+      // last option is the default sa.
+      pod = if (Option(pod.pod.getSpec.getServiceAccount).isEmpty) {
+        buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod)
+      } else {
+        pod.pod
+      })
+  }
+  override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty

Review comment:
       Thank you for adding this. BTW, could you add one empty line before line 38?




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.k8s.features
+
+import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod}
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME
+import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount
+
+private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_])
+  extends KubernetesFeatureConfigStep {
+  private lazy val driverServiceAccount =
+    kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME)
+
+  override def configurePod(pod: SparkPod): SparkPod = {
+    pod.copy(
+      // if not setup by the pod template fallback to the driver's sa,
+      // last option is the default sa.
+      pod = if (Option(pod.pod.getSpec.getServiceAccount).isEmpty) {
+        buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod)
+      } else {
+        pod.pod
+      })
+  }
+  override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty
+
+  override def getAdditionalKubernetesResources(): Seq[io.fabric8.kubernetes.api.model.HasMetadata]

Review comment:
       Please import `io.fabric8.kubernetes.api.model.HasMetadata`.




----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   @dongjoon-hyun @erikerlandson please let me know if you have additional comments that I can address.


----------------------------------------------------------------
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] erikerlandson commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   ok to test


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   **[Test build #129467 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129467/testReport)** for PR 29877 at commit [`dafd0ac`](https://github.com/apache/spark/commit/dafd0ace9ffd942e89d04dd90e7564bd717983ba).


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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] erikerlandson commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   I'm OK to merge if @dongjoon-hyun  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 removed a comment on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.k8s.features
+
+import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod}
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME
+import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount
+
+private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_])
+  extends KubernetesFeatureConfigStep {
+  private lazy val driverServiceAccount =
+    kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME)
+
+  override def configurePod(pod: SparkPod): SparkPod = {
+    pod.copy(
+      // if not setup by the pod template fallback to the driver's sa,
+      // last option is the default sa.
+      pod = if (Option(pod.pod.getSpec.getServiceAccount).isEmpty) {
+        buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod)
+      } else {
+        pod.pod
+      })
+  }
+  override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty
+
+  override def getAdditionalKubernetesResources(): Seq[io.fabric8.kubernetes.api.model.HasMetadata]
+  = Seq.empty

Review comment:
       Then, you don't need break a line like this.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Can one of the admins verify this patch?


----------------------------------------------------------------
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] erikerlandson commented on a change in pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
##########
@@ -61,10 +61,9 @@ private[spark] object Config extends Logging {
       .stringConf
       .createOptional
 
-  val KUBERNETES_AUTH_DRIVER_CONF_PREFIX =
-      "spark.kubernetes.authenticate.driver"
-  val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
-      "spark.kubernetes.authenticate.driver.mounted"
+  val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver"
+  val KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX = "spark.kubernetes.authenticate.executor"
+  val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted"

Review comment:
       this may blow the 100-char limit, if I counted right




----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   @dongjoon-hyun  could we merge this if there aren't additional comments or feedback?


----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   Thanks @dongjoon-hyun. 


----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   Thanks @dongjoon-hyun. I shall address the comments in upcoming commits.


----------------------------------------------------------------
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] nssalian commented on pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   > ok to test
   
   We might need an admin to initiate tests again. ping @dongjoon-hyun 


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala
##########
@@ -60,4 +64,15 @@ private[spark] object KubernetesUtils {
   }
 
   def parseMasterUrl(url: String): String = url.substring("k8s://".length)
+
+  def buildPodWithServiceAccount(serviceAccount: Option[String], pod: SparkPod): Option[Pod] = {
+      serviceAccount.map { account =>

Review comment:
       We are using `2-space` indentation.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


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


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


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


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala
##########
@@ -60,4 +64,15 @@ private[spark] object KubernetesUtils {
   }
 
   def parseMasterUrl(url: String): String = url.substring("k8s://".length)
+
+  def buildPodWithServiceAccount(serviceAccount: Option[String], pod: SparkPod): Option[Pod] = {
+      serviceAccount.map { account =>
+        new PodBuilder(pod.pod)
+          .editOrNewSpec()
+          .withServiceAccount(account)
+          .withServiceAccountName(account)
+          .endSpec()

Review comment:
       Please keep the original formatting.
   ```scala
   .editOrNewSpec()
     .withServiceAccount(account)
     .withServiceAccountName(account)
   .endSpec()
   ```




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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






----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency backport to branch-2.4

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


   Retest this please


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   **[Test build #129525 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129525/testReport)** for PR 29877 at commit [`489fa25`](https://github.com/apache/spark/commit/489fa25bf0cb8d23802838178fe38b894b694b6c).


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


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


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala
##########
@@ -18,12 +18,17 @@ package org.apache.spark.scheduler.cluster.k8s
 
 import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod}
 import org.apache.spark.deploy.k8s.features._
-import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep}
+import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep,
+  ExecutorKubernetesCredentialsFeatureStep,
+LocalDirsFeatureStep, MountSecretsFeatureStep}
 
 private[spark] class KubernetesExecutorBuilder(
     provideBasicStep: (KubernetesConf [KubernetesExecutorSpecificConf])
       => BasicExecutorFeatureStep =
       new BasicExecutorFeatureStep(_),
+    provideCredentialsStep: (KubernetesConf [KubernetesExecutorSpecificConf])
+      => ExecutorKubernetesCredentialsFeatureStep =
+    new ExecutorKubernetesCredentialsFeatureStep(_),

Review comment:
       indentation?




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala
##########
@@ -84,6 +84,13 @@ private[spark] trait BasicTestsSuite { k8sSuite: KubernetesSuite =>
       })
   }
 
+  test("All pods have the same service account by default", k8sTestTag) {
+      runSparkPiAndVerifyCompletion(

Review comment:
       indentation?




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.k8s.features
+
+import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod}
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME
+import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount
+
+private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_])
+  extends KubernetesFeatureConfigStep {
+  private lazy val driverServiceAccount =
+    kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME)

Review comment:
       ditto. Please keep one-liner (the same format with the original code of `master` branch).




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.k8s.features
+
+import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod}
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME
+import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount
+
+private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_])
+  extends KubernetesFeatureConfigStep {
+  private lazy val driverServiceAccount =
+    kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME)

Review comment:
       ditto. Please keep one-liner (the same format with the original code of `master` branch) if that doesn't break Scalastyle.




----------------------------------------------------------------
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 closed pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #29877:
URL: https://github.com/apache/spark/pull/29877


   


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala
##########
@@ -18,12 +18,17 @@ package org.apache.spark.scheduler.cluster.k8s
 
 import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod}
 import org.apache.spark.deploy.k8s.features._
-import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep}
+import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep,
+  ExecutorKubernetesCredentialsFeatureStep,
+LocalDirsFeatureStep, MountSecretsFeatureStep}

Review comment:
       ditto.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   **[Test build #129525 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129525/testReport)** for PR 29877 at commit [`489fa25`](https://github.com/apache/spark/commit/489fa25bf0cb8d23802838178fe38b894b694b6c).


----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala
##########
@@ -50,8 +55,10 @@ private[spark] class KubernetesExecutorBuilder(
     val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) {
       Seq(provideVolumesStep(kubernetesConf))
     } else Nil
+    val credentialsFeatures = Seq(provideCredentialsStep(kubernetesConf))

Review comment:
       `credentialsFeatures` -> `credentialsFeature`.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   **[Test build #129467 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129467/testReport)** for PR 29877 at commit [`dafd0ac`](https://github.com/apache/spark/commit/dafd0ace9ffd942e89d04dd90e7564bd717983ba).
    * 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] dongjoon-hyun commented on a change in pull request #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala
##########
@@ -18,12 +18,17 @@ package org.apache.spark.scheduler.cluster.k8s
 
 import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod}
 import org.apache.spark.deploy.k8s.features._
-import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep}
+import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep,
+  ExecutorKubernetesCredentialsFeatureStep,

Review comment:
       Please don't break `import` statement. `import` is excluded at line-length style check.




----------------------------------------------------------------
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 #29877: [SPARK-27872][K8S][2.4] Fix executor service account inconsistency

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


   Got it. Thank you for updates.


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