[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-28 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/19468


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-28 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153427945
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

@felixcheung That would be a behavior change w.r.t how we have used docker 
till now in spark - we expect users to explicitly specify the image.
We should not be introducing default behavior for a new feature without 
adequately understanding the implications of it - we can always introduce a 
smart default in a future spark version as required with minimal impact to end 
users.

I do agree that it is a good idea to validate version, etc - irrespective 
of whether we have a default value or not. It will help not just k8s 
integration, but also mesos and spark on yarn + docker.

For now, we can fail fast when value is not specified.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153401411
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

IMO we should, as a part of releasing Spark with this feature, to have 
Docker image in the same release. Granted, it is not a guarantee, but given 
what we know about building Docker images and Docker image publishing supported 
officially by the ASF / ASF Infra, this shouldn't be a problem.

On the flip side if we couldn't have "official" or default images, then we 
should consider how to build some validations in the startup path such that 
"wrong" or mismatch images can be detected quickly / fail-fast.



---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153400419
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

The concern is the availability of this default image. If we can make a 
strong guarantee by making tagging and publication of the images part of the 
release process, this will not be an issue. @mridulm @foxish @mccheah thoughts?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153399402
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

I'm somewhat concerned about this though.
IMO one of key value we delivery with this feature is running on k8s out of 
the box, and the Docker images we are going released with Spark is a key aspect 
of that (again, as discussed, we should highlight that this SPIP/feature is 
going to deliver and release that)

Secondly, it is going to be very easy to make a mistake and set the "wrong" 
Docker images (there are many many ways this could go wrong, including mismatch 
versions of submission client vs driver &/ executor) that would be very hard to 
diagnose (by no means it will fail fast - worse case you get some subtle 
correctness issues in data output). And IMO having this version number set 
consistently by default is really going to help that.

What's the concern with having the default value?




---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153360441
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
+ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
+  .doc("Maximum number of attempts allowed for checking the reason of 
an executor loss " +
+"before it is assumed that the executor failed.")
+  .intConf
+  .checkValue(value => value > 0, "Maximum attempts of checks of 
executor lost reason " +
+"must be a positive integer")
+  .createWithDefault(5)
--- End diff --

Sure, changed back to 10.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153360149
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
+ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
+  .doc("Maximum number of attempts allowed for checking the reason of 
an executor loss " +
+"before it is assumed that the executor failed.")
+  .intConf
+  .checkValue(value => value > 0, "Maximum attempts of checks of 
executor lost reason " +
+"must be a positive integer")
+  .createWithDefault(5)
--- End diff --

We should probably keep it the same as before, as it's what we've been 
running with already.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153359853
  
--- Diff: docs/configuration.md ---
@@ -1397,10 +1397,10 @@ Apart from these, the following properties are also 
available, and may be useful
 
 
   spark.scheduler.minRegisteredResourcesRatio
-  0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained 
mode
+  2.3.0 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone 
mode and Mesos coarse-grained mode
--- End diff --

Ah, totally mis-interpreted it. Fixed.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153359582
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
+ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
+  .doc("Maximum number of attempts allowed for checking the reason of 
an executor loss " +
+"before it is assumed that the executor failed.")
+  .intConf
+  .checkValue(value => value > 0, "Maximum attempts of checks of 
executor lost reason " +
+"must be a positive integer")
+  .createWithDefault(5)
--- End diff --

Yes, but I think 5 is a more sensible default than 10. @mccheah @foxish 
WDYT?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153359286
  
--- Diff: docs/configuration.md ---
@@ -1397,10 +1397,10 @@ Apart from these, the following properties are also 
available, and may be useful
 
 
   spark.scheduler.minRegisteredResourcesRatio
-  0.8 for YARN mode; 0.0 for standalone mode and Mesos coarse-grained 
mode
+  2.3.0 for KUBERNETES mode; 0.8 for YARN mode; 0.0 for standalone 
mode and Mesos coarse-grained mode
--- End diff --

`0.8 for KUBERNETES mode` ?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153359123
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153359040
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -111,5 +111,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS =
+ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts")
+  .doc("Maximum number of attempts allowed for checking the reason of 
an executor loss " +
+"before it is assumed that the executor failed.")
+  .intConf
+  .checkValue(value => value > 0, "Maximum attempts of checks of 
executor lost reason " +
+"must be a positive integer")
+  .createWithDefault(5)
--- End diff --

Wasn't this not 10 earlier ?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153357523
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153356659
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153354379
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r15335
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153351876
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,456 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+
+import scala.collection.{concurrent, mutable}
+import scala.collection.JavaConverters._
+import scala.concurrent.{ExecutionContext, Future}
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.config._
+import org.apache.spark.deploy.k8s.constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK.
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  // Indexed by executor pod names and guarded by 
RUNNING_EXECUTOR_PODS_LOCK.
+  private val runningPodsToExecutors = new mutable.HashMap[String, String]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(
+  throw new SparkException("Must specify the driver pod name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+  requestExecutorsService)
+
+  private val driverPod = try {
+kubernetesClient.pods()
+  .inNamespace(kubernetesNamespace)
+  .withName(kubernetesDriverPodName)
+  .get()
+  } catch {
+case throwable: Throwable =>
--- End diff --

Never mind, am looking at an older commit.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153351573
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,456 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+
+import scala.collection.{concurrent, mutable}
+import scala.collection.JavaConverters._
+import scala.concurrent.{ExecutionContext, Future}
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.config._
+import org.apache.spark.deploy.k8s.constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK.
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  // Indexed by executor pod names and guarded by 
RUNNING_EXECUTOR_PODS_LOCK.
+  private val runningPodsToExecutors = new mutable.HashMap[String, String]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(
+  throw new SparkException("Must specify the driver pod name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+  requestExecutorsService)
+
+  private val driverPod = try {
+kubernetesClient.pods()
+  .inNamespace(kubernetesNamespace)
+  .withName(kubernetesDriverPodName)
+  .get()
+  } catch {
+case throwable: Throwable =>
--- End diff --

@liyinan926 this is outstanding, can we remove the catch here?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153350771
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153350064
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153349685
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153349561
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

I agree.  Removed in 
https://github.com/apache/spark/pull/19468/commits/4bed817e5ab1a26e70050254fec453e839af4c8c.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread foxish
Github user foxish commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153341501
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts =

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153341275
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153335335
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153334988
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r15958
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153332841
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153332673
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153332360
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153331617
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153331071
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153330144
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153329128
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153327541
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153327654
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.File
+
+import io.fabric8.kubernetes.client.Config
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.{ExternalClusterManager, 
SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+import org.apache.spark.util.ThreadUtils
+
+private[spark] class KubernetesClusterManager extends 
ExternalClusterManager with Logging {
+
+  override def canCreate(masterURL: String): Boolean = 
masterURL.startsWith("k8s")
+
+  override def createTaskScheduler(sc: SparkContext, masterURL: String): 
TaskScheduler = {
+new TaskSchedulerImpl(sc)
+  }
+
+  override def createSchedulerBackend(
+  sc: SparkContext,
+  masterURL: String,
+  scheduler: TaskScheduler): SchedulerBackend = {
+val sparkConf = sc.getConf
+
+val kubernetesClient = 
SparkKubernetesClientFactory.createKubernetesClient(
+  KUBERNETES_MASTER_INTERNAL_URL,
+  Some(sparkConf.get(KUBERNETES_NAMESPACE)),
+  APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX,
+  sparkConf,
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)),
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)))
+
+val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf)
+val allocatorExecutor = ThreadUtils
+  .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator")
+val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool(
+  "kubernetes-executor-requests")
--- End diff --

Sounds good.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153326260
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153325898
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Kubernetes image pull policy. Valid values are Always, Never, 
and IfNotPresent.")
+  .stringConf
+  .checkValues(Set("Always", "Never", "IfNotPresent"))
+  .createWithDefault("IfNotPresent")
+
+  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver"
+  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver.mounted"
+  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
+  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
+  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
+  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
+  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
+
+  val KUBERNETES_SERVICE_ACCOUNT_NAME =
+ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
+  .doc("Service account that is used when running the driver pod. The 
driver pod uses" +
+" this service account when requesting executor pods from the API 
server. If specific" +
+" credentials are given for the driver pod to use, the driver will 
favor" +
+" using those credentials instead.")
+  .stringConf
+  .createOptional
+
+  // Note that while we set a default for this when we start up the
+  // scheduler, the specific default value is dynamically determined
+  // based on the executor memory.
+  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
+ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
+  .doc("The amount of off-heap memory (in megabytes) to be allocated 
per executor. This" +
+" is memory that accounts for things like VM overheads, interned 
strings, other native" +
+" overheads, etc. This tends to grow with the executor size. 
(typically 6-10%).")
+  .bytesConf(ByteUnit.MiB)
+  .createOptional
--- End diff --

Perfect !


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153305536
  
--- Diff: resource-managers/kubernetes/core/pom.xml ---
@@ -0,0 +1,94 @@
+
+
+http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.3.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes_2.11
+  jar
+  Spark Project Kubernetes
+  
+kubernetes
+3.0.0
+  
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  io.fabric8
+  kubernetes-client
--- End diff --

Listed in f8e3249.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153305473
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.{ConfigBuilder, 
DefaultKubernetesClient, KubernetesClient}
+import io.fabric8.kubernetes.client.utils.HttpClientUtils
+import okhttp3.Dispatcher
--- End diff --

Fixed in f8e3249.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153303696
  
--- Diff: resource-managers/kubernetes/core/pom.xml ---
@@ -0,0 +1,94 @@
+
+
+http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.3.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes_2.11
+  jar
+  Spark Project Kubernetes
+  
+kubernetes
+3.0.0
+  
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  io.fabric8
+  kubernetes-client
--- End diff --

The dependency needs to be added to `NOTICE`, right?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread foxish
Github user foxish commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153297620
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

I think we should remove this instead of defaulting. WDYT @mccheah 


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread foxish
Github user foxish commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153296815
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts =

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153294451
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153282723
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153282383
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153278119
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153277116
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
--- End diff --

Done.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153272632
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -0,0 +1,50 @@
+/*
+ * 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
+
+private[spark] object Constants {
+
+  // Labels
+  val SPARK_APP_ID_LABEL = "spark-app-selector"
+  val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id"
+  val SPARK_ROLE_LABEL = "spark-role"
+  val SPARK_POD_DRIVER_ROLE = "driver"
+  val SPARK_POD_EXECUTOR_ROLE = "executor"
+
+  // Default and fixed ports
+  val DEFAULT_DRIVER_PORT = 7078
+  val DEFAULT_BLOCKMANAGER_PORT = 7079
+  val BLOCK_MANAGER_PORT_NAME = "blockmanager"
+  val EXECUTOR_PORT_NAME = "executor"
+
+  // Environment Variables
+  val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT"
+  val ENV_DRIVER_URL = "SPARK_DRIVER_URL"
+  val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES"
+  val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY"
+  val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID"
+  val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID"
+  val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP"
+  val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH"
+  val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH"
+  val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_"
--- End diff --

What do you mean by shared?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153269639
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.File
+
+import io.fabric8.kubernetes.client.Config
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.{ExternalClusterManager, 
SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+import org.apache.spark.util.ThreadUtils
+
+private[spark] class KubernetesClusterManager extends 
ExternalClusterManager with Logging {
+
+  override def canCreate(masterURL: String): Boolean = 
masterURL.startsWith("k8s")
+
+  override def createTaskScheduler(sc: SparkContext, masterURL: String): 
TaskScheduler = {
+new TaskSchedulerImpl(sc)
+  }
+
+  override def createSchedulerBackend(
+  sc: SparkContext,
+  masterURL: String,
+  scheduler: TaskScheduler): SchedulerBackend = {
+val sparkConf = sc.getConf
+
+val kubernetesClient = 
SparkKubernetesClientFactory.createKubernetesClient(
+  KUBERNETES_MASTER_INTERNAL_URL,
+  Some(sparkConf.get(KUBERNETES_NAMESPACE)),
+  APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX,
+  sparkConf,
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)),
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)))
+
+val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf)
+val allocatorExecutor = ThreadUtils
+  .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator")
+val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool(
+  "kubernetes-executor-requests")
--- End diff --

The executor services are passed in to make 
`KubernetesClusterSchedulerBackend` more unit testable. In 
`KubernetesClusterSchedulerBackendSuite`, the executor services are mocked. 


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153248875
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153246728
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Kubernetes image pull policy. Valid values are Always, Never, 
and IfNotPresent.")
+  .stringConf
+  .checkValues(Set("Always", "Never", "IfNotPresent"))
+  .createWithDefault("IfNotPresent")
+
+  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver"
+  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver.mounted"
+  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
+  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
+  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
+  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
+  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
+
+  val KUBERNETES_SERVICE_ACCOUNT_NAME =
+ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
+  .doc("Service account that is used when running the driver pod. The 
driver pod uses" +
+" this service account when requesting executor pods from the API 
server. If specific" +
+" credentials are given for the driver pod to use, the driver will 
favor" +
+" using those credentials instead.")
+  .stringConf
+  .createOptional
+
+  // Note that while we set a default for this when we start up the
+  // scheduler, the specific default value is dynamically determined
+  // based on the executor memory.
+  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
+ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
+  .doc("The amount of off-heap memory (in megabytes) to be allocated 
per executor. This" +
+" is memory that accounts for things like VM overheads, interned 
strings, other native" +
+" overheads, etc. This tends to grow with the executor size. 
(typically 6-10%).")
+  .bytesConf(ByteUnit.MiB)
+  .createOptional
--- End diff --

It's in #19717. See 
https://github.com/apache-spark-on-k8s/spark/blob/60234a29846955b8a6e8cb6fbb1dc35da3c3b4f2/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153243729
  
--- 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)
--- End diff --

This is only used to set the environment variable `SPARK_EXECUTOR_PORT` in 
the executor container, which then gets used to set `spark.executor.port` for 
the executor. If it's no longer used by Spark, it's not required.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153242284
  
--- Diff: resource-managers/kubernetes/core/pom.xml ---
@@ -0,0 +1,94 @@
+
+
+http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.3.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes_2.11
+  jar
+  Spark Project Kubernetes
+  
+kubernetes
+3.0.0
--- End diff --

We have been using version 3.0.0 in our forked repo for a couple of months 
now. We haven't seen any issue with it.  


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153240026
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-27 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153239467
  
--- 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-$execu

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153088979
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala
 ---
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.File
+
+import io.fabric8.kubernetes.client.Config
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.{ExternalClusterManager, 
SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+import org.apache.spark.util.ThreadUtils
+
+private[spark] class KubernetesClusterManager extends 
ExternalClusterManager with Logging {
+
+  override def canCreate(masterURL: String): Boolean = 
masterURL.startsWith("k8s")
+
+  override def createTaskScheduler(sc: SparkContext, masterURL: String): 
TaskScheduler = {
+new TaskSchedulerImpl(sc)
+  }
+
+  override def createSchedulerBackend(
+  sc: SparkContext,
+  masterURL: String,
+  scheduler: TaskScheduler): SchedulerBackend = {
+val sparkConf = sc.getConf
+
+val kubernetesClient = 
SparkKubernetesClientFactory.createKubernetesClient(
+  KUBERNETES_MASTER_INTERNAL_URL,
+  Some(sparkConf.get(KUBERNETES_NAMESPACE)),
+  APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX,
+  sparkConf,
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)),
+  Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)))
+
+val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf)
+val allocatorExecutor = ThreadUtils
+  .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator")
+val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool(
+  "kubernetes-executor-requests")
--- End diff --

Why is this service passed from outside ?
It is an impl detail of `KubernetesClusterSchedulerBackend` and should be 
initialized/managed within (create in constructor, shutdown in stop)


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153088232
  
--- 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)
--- End diff --

Is this required ? IIRC as of 2.0 the config was removed/is not longer used.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153091143
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153084210
  
--- Diff: resource-managers/kubernetes/core/pom.xml ---
@@ -0,0 +1,94 @@
+
+
+http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.3.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes_2.11
+  jar
+  Spark Project Kubernetes
+  
+kubernetes
+3.0.0
--- End diff --

Last time I asked about the client version, there were concerns regarding 
maturity/stability of 3.x compared to the 2.2 (iirc) version which was in use - 
where they resolved ?
I, obviously, prefer moving to the latest major version; but want to 
understand the risks involved - if any.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153087234
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153089366
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153089805
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153089664
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts 

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153087550
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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, SparkException}
+import org.apache.spark.deploy.k8s.ConfigurationUtils
+import org.apache.spark.deploy.k8s.config._
+import org.apache.spark.deploy.k8s.constants._
+import org.apache.spark.util.Utils
+
+/**
+ * Configures executor pods. Construct one of these with a SparkConf to 
set up properties that are
+ * common across all executors. Then, pass in dynamic parameters into 
createExecutorPod.
+ */
+private[spark] trait ExecutorPodFactory {
+  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 executorJarsDownloadDir = 
sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
+
+  private val executorLabels = 
ConfigurationUtils.parsePrefixedKeyValuePairs(
+sparkConf,
+KUBERNETES_EXECUTOR_LABEL_PREFIX,
+"executor label")
+  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" +
+  s" Spark.")
+
+  private val executorAnnotations =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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 kubernetesDriverPodName = sparkConf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+
+  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", 
1d)
+  private val executorLimitCores = 
sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
+
+  override def createExecutorPod(
+  executorId: String,

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153084631
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
--- End diff --

Should this default `spark-executor:$SPARK_VERSION` be present ? I would 
expect the image name to be always explicitly specified by admin (in 
spark-defaults) or overridden by user.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153089121
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,432 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
--- End diff --

We need to update docs/configuration.md to specify this is used by both 
yarn and kubernetes now.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153084513
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,115 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Kubernetes image pull policy. Valid values are Always, Never, 
and IfNotPresent.")
+  .stringConf
+  .checkValues(Set("Always", "Never", "IfNotPresent"))
+  .createWithDefault("IfNotPresent")
+
+  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver"
+  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver.mounted"
+  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
+  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
+  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
+  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
+  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
+
+  val KUBERNETES_SERVICE_ACCOUNT_NAME =
+ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
+  .doc("Service account that is used when running the driver pod. The 
driver pod uses" +
+" this service account when requesting executor pods from the API 
server. If specific" +
+" credentials are given for the driver pod to use, the driver will 
favor" +
+" using those credentials instead.")
+  .stringConf
+  .createOptional
+
+  // Note that while we set a default for this when we start up the
+  // scheduler, the specific default value is dynamically determined
+  // based on the executor memory.
+  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
+ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
+  .doc("The amount of off-heap memory (in megabytes) to be allocated 
per executor. This" +
+" is memory that accounts for things like VM overheads, interned 
strings, other native" +
+" overheads, etc. This tends to grow with the executor size. 
(typically 6-10%).")
+  .bytesConf(ByteUnit.MiB)
+  .createOptional
--- End diff --

What about driver memory overhead ?
I see that mesos does not support it, while yarn does - is it relevant here 
?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153090634
  
--- 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-$executor

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r15308
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.{ConfigBuilder, 
DefaultKubernetesClient, KubernetesClient}
+import io.fabric8.kubernetes.client.utils.HttpClientUtils
+import okhttp3.Dispatcher
--- End diff --

and license..


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153083057
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.{ConfigBuilder, 
DefaultKubernetesClient, KubernetesClient}
+import io.fabric8.kubernetes.client.utils.HttpClientUtils
+import okhttp3.Dispatcher
--- End diff --

this seems new, should this be listed in pom file?



---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153083018
  
--- Diff: resource-managers/kubernetes/core/pom.xml ---
@@ -0,0 +1,94 @@
+
+
+http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  4.0.0
+  
+org.apache.spark
+spark-parent_2.11
+2.3.0-SNAPSHOT
+../../../pom.xml
+  
+
+  spark-kubernetes_2.11
+  jar
+  Spark Project Kubernetes
+  
+kubernetes
+3.0.0
+  
+
+  
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+
+
+
+  org.apache.spark
+  spark-core_${scala.binary.version}
+  ${project.version}
+  test-jar
+  test
+
+
+
+  io.fabric8
+  kubernetes-client
--- End diff --

these dependencies need to be listed, please see
http://www.apache.org/dev/licensing-howto.html



---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-26 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r153082500
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala
 ---
@@ -0,0 +1,50 @@
+/*
+ * 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
+
+private[spark] object Constants {
+
+  // Labels
+  val SPARK_APP_ID_LABEL = "spark-app-selector"
+  val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id"
+  val SPARK_ROLE_LABEL = "spark-role"
+  val SPARK_POD_DRIVER_ROLE = "driver"
+  val SPARK_POD_EXECUTOR_ROLE = "executor"
+
+  // Default and fixed ports
+  val DEFAULT_DRIVER_PORT = 7078
+  val DEFAULT_BLOCKMANAGER_PORT = 7079
+  val BLOCK_MANAGER_PORT_NAME = "blockmanager"
+  val EXECUTOR_PORT_NAME = "executor"
+
+  // Environment Variables
+  val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT"
+  val ENV_DRIVER_URL = "SPARK_DRIVER_URL"
+  val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES"
+  val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY"
+  val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID"
+  val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID"
+  val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP"
+  val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH"
+  val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH"
+  val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_"
--- End diff --

nit: these const string should probably be shared


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152634879
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,429 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152634018
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,429 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCoun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152633977
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Docker image pull policy when pulling any docker image in 
Kubernetes integration")
+  .stringConf
+  .createWithDefault("IfNotPresent")
+
+  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver"
+  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver.mounted"
+  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
+  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
+  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
+  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
+  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
+
+  val KUBERNETES_SERVICE_ACCOUNT_NAME =
+ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
+  .doc("Service account that is used when running the driver pod. The 
driver pod uses" +
+" this service account when requesting executor pods from the API 
server. If specific" +
+" credentials are given for the driver pod to use, the driver will 
favor" +
+" using those credentials instead.")
+  .stringConf
+  .createOptional
+
+  // Note that while we set a default for this when we start up the
+  // scheduler, the specific default value is dynamically determined
+  // based on the executor memory.
+  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
+ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
+  .doc("The amount of off-heap memory (in megabytes) to be allocated 
per executor. This" +
+" is memory that accounts for things like VM overheads, interned 
strings, other native" +
+" overheads, etc. This tends to grow with the executor size. 
(typically 6-10%).")
+  .bytesConf(ByteUnit.MiB)
+  .createOptional
+
+  val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
+  val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = 
"spark.kubernetes.executor.annotation."
+
+  val KUBERNETES_DRIVER_POD_NAME =
+ConfigBuilder("spark.kubernetes.driver.pod.name")
+  .doc("Name of the driver pod.")
+  .stringConf
+  .createOptional
+
+  val KUBERNETES_EXECUTOR_POD_NAME_PREFIX =
+ConfigBuilder("spark.kubernetes.executor.podNamePrefix")
+  .doc("Prefix to use in front of the executor pod names.")
+  .internal()
+  .stringConf
+  .createWithDefault("spark")
+
+  val KUBERNETES_ALLOCATION_BATCH_SIZE =
+ConfigBuilder("spark.kubernetes.allocation.batch.size")
+  .doc("Number of pods to launch at once in each roun

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152633953
  
--- 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.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
--- End diff --

Done.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152633990
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Docker image pull policy when pulling any docker image in 
Kubernetes integration")
--- End diff --

Done.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152633966
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.{ConfigBuilder, 
DefaultKubernetesClient, KubernetesClient}
+import io.fabric8.kubernetes.client.utils.HttpClientUtils
+import okhttp3.Dispatcher
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus 
common suffixes to
+ * parse configuration keys, similar to the manner in which Spark's 
SecurityManager parses SSL
+ * options for different components.
+ */
+private[spark] object SparkKubernetesClientFactory {
+
+  def createKubernetesClient(
+  master: String,
+  namespace: Option[String],
+  kubernetesAuthConfPrefix: String,
+  sparkConf: SparkConf,
+  defaultServiceAccountToken: Option[File],
+  defaultServiceAccountCaCert: Option[File]): KubernetesClient = {
+val oauthTokenFileConf = 
s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
+val oauthTokenConf = 
s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
+val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
+  .map(new File(_))
+  .orElse(defaultServiceAccountToken)
+val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
+ConfigurationUtils.requireNandDefined(
+  oauthTokenFile,
+  oauthTokenValue,
+  s"Cannot specify OAuth token through both a file $oauthTokenFileConf 
and a " +
+s"value $oauthTokenConf.")
+
+val caCertFile = sparkConf
+  .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
+  .orElse(defaultServiceAccountCaCert.map(_.getAbsolutePath))
+val clientKeyFile = sparkConf
+  .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
+val clientCertFile = sparkConf
+  
.getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
+val dispatcher = new Dispatcher(
+  ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
+val config = new ConfigBuilder()
+  .withApiVersion("v1")
+  .withMasterUrl(master)
+  .withWebsocketPingInterval(0)
+  .withOption(oauthTokenValue) {
+(token, configBuilder) => configBuilder.withOauthToken(token)
+  }.withOption(oauthTokenFile) {
+(file, configBuilder) =>
+configBuilder.withOauthToken(Files.toString(file, 
Charsets.UTF_8))
+  }.withOption(caCertFile) {
+(file, configBuilder) => configBuilder.withCaCertFile(file)
+  }.withOption(clientKeyFile) {
+(file, configBuilder) => configBuilder.withClientKeyFile(file)
+  }.withOption(clientCertFile) {
+(file, configBuilder) => configBuilder.withClientCertFile(file)
+  }.withOption(namespace) {
+(ns, configBuilder) => configBuilder.withNamespace(ns)
+  }.build()
+val baseHttpClient = HttpClientUtils.createHttpClient(config)
+val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
+  .dispatcher(dispatcher)
+  .build()
+new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
+  }
+
+  private implicit class OptionConfigurableConfigBuilder(val 
configBuilder: ConfigBuilder)
+extends AnyVal {
+
+def withOption[T]
+(option: Option[T])
+(configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = 
{
--- End diff --

Done.


---

-

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152631296
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

Actually we use that to set environment variable `SPARK_EXECUTOR_MEMORY` in 
the executor container, which is then used to set `spark.executor.memory`.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152522621
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,429 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts =

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152505563
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.{ConfigBuilder, 
DefaultKubernetesClient, KubernetesClient}
+import io.fabric8.kubernetes.client.utils.HttpClientUtils
+import okhttp3.Dispatcher
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus 
common suffixes to
+ * parse configuration keys, similar to the manner in which Spark's 
SecurityManager parses SSL
+ * options for different components.
+ */
+private[spark] object SparkKubernetesClientFactory {
+
+  def createKubernetesClient(
+  master: String,
+  namespace: Option[String],
+  kubernetesAuthConfPrefix: String,
+  sparkConf: SparkConf,
+  defaultServiceAccountToken: Option[File],
+  defaultServiceAccountCaCert: Option[File]): KubernetesClient = {
+val oauthTokenFileConf = 
s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX"
+val oauthTokenConf = 
s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX"
+val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf)
+  .map(new File(_))
+  .orElse(defaultServiceAccountToken)
+val oauthTokenValue = sparkConf.getOption(oauthTokenConf)
+ConfigurationUtils.requireNandDefined(
+  oauthTokenFile,
+  oauthTokenValue,
+  s"Cannot specify OAuth token through both a file $oauthTokenFileConf 
and a " +
+s"value $oauthTokenConf.")
+
+val caCertFile = sparkConf
+  .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX")
+  .orElse(defaultServiceAccountCaCert.map(_.getAbsolutePath))
+val clientKeyFile = sparkConf
+  .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX")
+val clientCertFile = sparkConf
+  
.getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX")
+val dispatcher = new Dispatcher(
+  ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher"))
+val config = new ConfigBuilder()
+  .withApiVersion("v1")
+  .withMasterUrl(master)
+  .withWebsocketPingInterval(0)
+  .withOption(oauthTokenValue) {
+(token, configBuilder) => configBuilder.withOauthToken(token)
+  }.withOption(oauthTokenFile) {
+(file, configBuilder) =>
+configBuilder.withOauthToken(Files.toString(file, 
Charsets.UTF_8))
+  }.withOption(caCertFile) {
+(file, configBuilder) => configBuilder.withCaCertFile(file)
+  }.withOption(clientKeyFile) {
+(file, configBuilder) => configBuilder.withClientKeyFile(file)
+  }.withOption(clientCertFile) {
+(file, configBuilder) => configBuilder.withClientCertFile(file)
+  }.withOption(namespace) {
+(ns, configBuilder) => configBuilder.withNamespace(ns)
+  }.build()
+val baseHttpClient = HttpClientUtils.createHttpClient(config)
+val httpClientWithCustomDispatcher = baseHttpClient.newBuilder()
+  .dispatcher(dispatcher)
+  .build()
+new DefaultKubernetesClient(httpClientWithCustomDispatcher, config)
+  }
+
+  private implicit class OptionConfigurableConfigBuilder(val 
configBuilder: ConfigBuilder)
+extends AnyVal {
+
+def withOption[T]
+(option: Option[T])
+(configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = 
{
--- End diff --

nit: indent (4 more spaces

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152521995
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala
 ---
@@ -0,0 +1,429 @@
+/*
+ * 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 java.io.Closeable
+import java.net.InetAddress
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, 
ScheduledExecutorService, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, 
AtomicReference}
+import javax.annotation.concurrent.GuardedBy
+
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, 
KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, Future}
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.k8s.Config._
+import org.apache.spark.deploy.k8s.Constants._
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, 
TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, 
SchedulerBackendUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesClusterSchedulerBackend(
+scheduler: TaskSchedulerImpl,
+rpcEnv: RpcEnv,
+executorPodFactory: ExecutorPodFactory,
+kubernetesClient: KubernetesClient,
+allocatorExecutor: ScheduledExecutorService,
+requestExecutorsService: ExecutorService)
+  extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
+
+  import KubernetesClusterSchedulerBackend._
+
+  private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+  private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+  @GuardedBy("RUNNING_EXECUTOR_PODS_LOCK")
+  private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+  private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]()
+  private val podsWithKnownExitReasons = new ConcurrentHashMap[String, 
ExecutorExited]()
+  private val disconnectedPodsByExecutorIdPendingRemoval = new 
ConcurrentHashMap[String, Pod]()
+
+  private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+
+  private val kubernetesDriverPodName = conf
+.get(KUBERNETES_DRIVER_POD_NAME)
+.getOrElse(throw new SparkException("Must specify the driver pod 
name"))
+  private implicit val requestExecutorContext = 
ExecutionContext.fromExecutorService(
+requestExecutorsService)
+
+  private val driverPod = kubernetesClient.pods()
+.inNamespace(kubernetesNamespace)
+.withName(kubernetesDriverPodName)
+.get()
+
+  protected override val minRegisteredRatio =
+if 
(conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+  0.8
+} else {
+  super.minRegisteredRatio
+}
+
+  private val executorWatchResource = new AtomicReference[Closeable]
+  private val totalExpectedExecutors = new AtomicInteger(0)
+
+  private val driverUrl = RpcEndpointAddress(
+conf.get("spark.driver.host"),
+conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+  private val initialExecutors = 
SchedulerBackendUtils.getInitialTargetExecutorNumber(conf)
+
+  private val podAllocationInterval = 
conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+
+  private val podAllocationSize = 
conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+
+  private val allocatorRunnable = new Runnable {
+
+// Maintains a map of executor id to count of checks performed to 
learn the loss reason
+// for an executor.
+private val executorReasonCheckAttemptCounts =

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152502008
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Docker image pull policy when pulling any docker image in 
Kubernetes integration")
--- End diff --

Can you add a list of acceptable values for this config if possible and add 
`.checkValues()` for them?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152509429
  
--- 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.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
--- End diff --

We can use `sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES)`?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152503394
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 ---
@@ -0,0 +1,114 @@
+/*
+ * 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
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object Config extends Logging {
+
+  val KUBERNETES_NAMESPACE =
+ConfigBuilder("spark.kubernetes.namespace")
+  .doc("The namespace that will be used for running the driver and 
executor pods. When using" +
+" spark-submit in cluster mode, this can also be passed to 
spark-submit via the" +
+" --kubernetes-namespace command line argument.")
+  .stringConf
+  .createWithDefault("default")
+
+  val EXECUTOR_DOCKER_IMAGE =
+ConfigBuilder("spark.kubernetes.executor.docker.image")
+  .doc("Docker image to use for the executors. Specify this using the 
standard Docker tag" +
+" format.")
+  .stringConf
+  .createWithDefault(s"spark-executor:$SPARK_VERSION")
+
+  val DOCKER_IMAGE_PULL_POLICY =
+ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+  .doc("Docker image pull policy when pulling any docker image in 
Kubernetes integration")
+  .stringConf
+  .createWithDefault("IfNotPresent")
+
+  val APISERVER_AUTH_DRIVER_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver"
+  val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX =
+  "spark.kubernetes.authenticate.driver.mounted"
+  val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken"
+  val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile"
+  val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile"
+  val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile"
+  val CA_CERT_FILE_CONF_SUFFIX = "caCertFile"
+
+  val KUBERNETES_SERVICE_ACCOUNT_NAME =
+ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName")
+  .doc("Service account that is used when running the driver pod. The 
driver pod uses" +
+" this service account when requesting executor pods from the API 
server. If specific" +
+" credentials are given for the driver pod to use, the driver will 
favor" +
+" using those credentials instead.")
+  .stringConf
+  .createOptional
+
+  // Note that while we set a default for this when we start up the
+  // scheduler, the specific default value is dynamically determined
+  // based on the executor memory.
+  val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD =
+ConfigBuilder("spark.kubernetes.executor.memoryOverhead")
+  .doc("The amount of off-heap memory (in megabytes) to be allocated 
per executor. This" +
+" is memory that accounts for things like VM overheads, interned 
strings, other native" +
+" overheads, etc. This tends to grow with the executor size. 
(typically 6-10%).")
+  .bytesConf(ByteUnit.MiB)
+  .createOptional
+
+  val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
+  val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = 
"spark.kubernetes.executor.annotation."
+
+  val KUBERNETES_DRIVER_POD_NAME =
+ConfigBuilder("spark.kubernetes.driver.pod.name")
+  .doc("Name of the driver pod.")
+  .stringConf
+  .createOptional
+
+  val KUBERNETES_EXECUTOR_POD_NAME_PREFIX =
+ConfigBuilder("spark.kubernetes.executor.podNamePrefix")
+  .doc("Prefix to use in front of the executor pod names.")
+  .internal()
+  .stringConf
+  .createWithDefault("spark")
+
+  val KUBERNETES_ALLOCATION_BATCH_SIZE =
+ConfigBuilder("spark.kubernetes.allocation.batch.size")
+  .doc("Number of pods to launch at once in each round of

[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-22 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152518072
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

so we don't need `executorMemoryString` now? 


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152458539
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

We are already translating it to k8s specific setting at line 117. For 
reference:

```
val executorMemoryQuantity = new QuantityBuilder(false)
  .withAmount(s"${executorMemoryMiB}Mi")
```


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152458560
  
--- Diff: 
resource-managers/kubernetes/core/src/test/resources/log4j.properties ---
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t 
%p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN
--- End diff --

Should we given this is for the unit tests?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread liyinan926
Github user liyinan926 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152458551
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala
 ---
@@ -0,0 +1,43 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+
+private[spark] object ConfigurationUtils {
+
+  /**
+   * Extract and parse Spark configuration properties with a given name 
prefix and
+   * return the result as a Map. Keys must not have more than one value.
+   *
+   * @param sparkConf Spark configuration
+   * @param prefix the given property name prefix
+   * @param configType a descriptive note on the type of entities of 
interest
+   * @return a Map storing the configuration property keys and values
+   */
+  def parsePrefixedKeyValuePairs(
+  sparkConf: SparkConf,
+  prefix: String,
+  configType: String): Map[String, String] = {
--- End diff --

We are not really using it in the context of this PR. Removed this 
parameter.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152450525
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

Maybe update the doc of this configuration to describe the difference while 
using Kubernetes backend?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152449333
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

`executorMemoryMiB` is an int value representing the numeber of megabytes. 
I think we should translate it to whatever k8s accepts as byte string, to keep 
the same semantic.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152448480
  
--- Diff: 
resource-managers/kubernetes/core/src/test/resources/log4j.properties ---
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t 
%p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN
--- End diff --

shall we also put these in `config/log4j.properties.template`?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152446177
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

Just for reference:

> Limits and requests for memory are measured in bytes. You can express 
memory as a plain integer or as a fixed-point integer using one of these 
suffixes: E, P, T, G, M, k. You can also use the power-of-two equivalents: Ei, 
Pi, Ti, Gi, Mi, Ki.


https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#meaning-of-memory



---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152445984
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

We bias towards keeping Spark configuration consistent across resource 
managers and then translating it to what Kubernetes expects. Should we instead 
be accepting Kubernetes-formatted memory strings and translating it on the 
environment variable?

Regardless, there is going to be a difference in the string we send to the 
JVM in `-Xmx` and `-Xms` and what we pass to Kubernetes. We could translate 
everything to just raw bytes to keep everything consistent.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152445535
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

IIUC, you are trying to bypass the Spark's byte string parsing and pass the 
byte string to k8s directly.

This may be confusing as `EXECUTOR_MEMORY` is an existing Spark config. 
E.g. when users specify `100mb` for this config, they expect to see executors 
having 100mb memory whatever the resource manager is. If k8s parses `100mb` 
differently, the behavior may be unexpected.


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152444685
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
 ---
@@ -0,0 +1,229 @@
+/*
+ * 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,
+"executor label")
+  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,
+  "executor annotation")
+  private val nodeSelector =
+ConfigurationUtils.parsePrefixedKeyValuePairs(
+  sparkConf,
+  KUBERNETES_NODE_SELECTOR_PREFIX,
+  "node selector")
+
+  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)
--- End diff --

`executorMemoryMiB` is the number of MB. I'm not sure what k8s expects, but 
passing `executorMemoryMiB + "MB"` should be pretty safe?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152443965
  
--- Diff: 
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala
 ---
@@ -0,0 +1,43 @@
+/*
+ * 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
+
+import org.apache.spark.SparkConf
+
+private[spark] object ConfigurationUtils {
+
+  /**
+   * Extract and parse Spark configuration properties with a given name 
prefix and
+   * return the result as a Map. Keys must not have more than one value.
+   *
+   * @param sparkConf Spark configuration
+   * @param prefix the given property name prefix
+   * @param configType a descriptive note on the type of entities of 
interest
+   * @return a Map storing the configuration property keys and values
+   */
+  def parsePrefixedKeyValuePairs(
+  sparkConf: SparkConf,
+  prefix: String,
+  configType: String): Map[String, String] = {
--- End diff --

where do we use this parameter?


---

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



[GitHub] spark pull request #19468: [SPARK-18278] [Scheduler] Spark on Kubernetes - B...

2017-11-21 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/19468#discussion_r152443488
  
--- Diff: pom.xml ---
@@ -2648,6 +2648,13 @@
   
 
 
+
+  kubernetes
+  
+resource-managers/kubernetes/core
--- End diff --

See https://github.com/apache/spark/pull/19468#discussion_r145250461


---

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



  1   2   3   4   >