Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19468#discussion_r153087493
  
    --- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
    @@ -0,0 +1,226 @@
    +/*
    + * 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.scheduler.cluster.k8s
    +
    +import scala.collection.JavaConverters._
    +
    +import io.fabric8.kubernetes.api.model._
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.deploy.k8s.Config._
    +import org.apache.spark.deploy.k8s.ConfigurationUtils
    +import org.apache.spark.deploy.k8s.Constants._
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A factory class for configuring and creating executor pods.
    + */
    +private[spark] trait ExecutorPodFactory {
    +
    +  /**
    +   * Configure and construct an executor pod with the given parameters.
    +   */
    +  def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod
    +}
    +
    +private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
    +  extends ExecutorPodFactory {
    +
    +  import ExecutorPodFactoryImpl._
    +
    +  private val executorExtraClasspath =
    +    sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
    +
    +  private val executorLabels = 
ConfigurationUtils.parsePrefixedKeyValuePairs(
    +    sparkConf,
    +    KUBERNETES_EXECUTOR_LABEL_PREFIX)
    +  require(
    +    !executorLabels.contains(SPARK_APP_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is 
reserved for Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it 
is reserved for" +
    +      " Spark.")
    +  require(
    +    !executorLabels.contains(SPARK_ROLE_LABEL),
    +    s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is 
reserved for Spark.")
    +
    +  private val executorAnnotations =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_EXECUTOR_ANNOTATION_PREFIX)
    +  private val nodeSelector =
    +    ConfigurationUtils.parsePrefixedKeyValuePairs(
    +      sparkConf,
    +      KUBERNETES_NODE_SELECTOR_PREFIX)
    +
    +  private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE)
    +  private val dockerImagePullPolicy = 
sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
    +  private val executorPort = sparkConf.getInt("spark.executor.port", 
DEFAULT_STATIC_PORT)
    +  private val blockManagerPort = sparkConf
    +    .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
    +
    +  private val executorPodNamePrefix = 
sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
    +
    +  private val executorMemoryMiB = 
sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
    +  private val executorMemoryString = sparkConf.get(
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
    +    org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
    +
    +  private val memoryOverheadMiB = sparkConf
    +    .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
    +    .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt,
    +      MEMORY_OVERHEAD_MIN_MIB))
    +  private val executorMemoryWithOverhead = executorMemoryMiB + 
memoryOverheadMiB
    +
    +  private val executorCores = sparkConf.getDouble("spark.executor.cores", 
1)
    +  private val executorLimitCores = 
sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)
    +
    +  override def createExecutorPod(
    +      executorId: String,
    +      applicationId: String,
    +      driverUrl: String,
    +      executorEnvs: Seq[(String, String)],
    +      driverPod: Pod,
    +      nodeToLocalTaskCount: Map[String, Int]): Pod = {
    +    val name = s"$executorPodNamePrefix-exec-$executorId"
    +
    +    // hostname must be no longer than 63 characters, so take the last 63 
characters of the pod
    +    // name as the hostname.  This preserves uniqueness since the end of 
name contains
    +    // executorId
    +    val hostname = name.substring(Math.max(0, name.length - 63))
    --- End diff --
    
    What is hostname used for here ?
    Based on previous discussion, there is no support for hostname and 
kubernetes support is only IP based - given this, why not simply set it to some 
arbitrary random string and not depend on `name` ? (The comment about 63 chars 
is still incorrect btw - hostnames go upto 255, labels go upto 63 : I think I 
mentioned this before).
    Or is this to keep the option open in future for dns support ?


---

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

Reply via email to