aboutsummaryrefslogtreecommitdiff
path: root/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
blob: b25253978258003b23a4e9a1e6469c7834a42292 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
/*
 * 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.executor

import java.nio.ByteBuffer

import scala.collection.JavaConverters._

import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver}
import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
import org.apache.mesos.protobuf.ByteString

import org.apache.spark.{SparkConf, SparkEnv, TaskState}
import org.apache.spark.TaskState
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.TaskDescription
import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerUtils
import org.apache.spark.util.Utils

private[spark] class MesosExecutorBackend
  extends MesosExecutor
  with MesosSchedulerUtils // TODO: fix
  with ExecutorBackend
  with Logging {

  var executor: Executor = null
  var driver: ExecutorDriver = null

  override def statusUpdate(taskId: Long, state: TaskState.TaskState, data: ByteBuffer) {
    val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build()
    driver.sendStatusUpdate(MesosTaskStatus.newBuilder()
      .setTaskId(mesosTaskId)
      .setState(taskStateToMesos(state))
      .setData(ByteString.copyFrom(data))
      .build())
  }

  override def registered(
      driver: ExecutorDriver,
      executorInfo: ExecutorInfo,
      frameworkInfo: FrameworkInfo,
      slaveInfo: SlaveInfo) {

    // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend.
    val cpusPerTask = executorInfo.getResourcesList.asScala
      .find(_.getName == "cpus")
      .map(_.getScalar.getValue.toInt)
      .getOrElse(0)
    val executorId = executorInfo.getExecutorId.getValue

    logInfo(s"Registered with Mesos as executor ID $executorId with $cpusPerTask cpus")
    this.driver = driver
    // Set a context class loader to be picked up by the serializer. Without this call
    // the serializer would default to the null class loader, and fail to find Spark classes
    // See SPARK-10986.
    Thread.currentThread().setContextClassLoader(this.getClass.getClassLoader)

    val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++
      Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue))
    val conf = new SparkConf(loadDefaults = true).setAll(properties)
    val port = conf.getInt("spark.executor.port", 0)
    val env = SparkEnv.createExecutorEnv(
      conf, executorId, slaveInfo.getHostname, port, cpusPerTask, None, isLocal = false)

    executor = new Executor(
      executorId,
      slaveInfo.getHostname,
      env)
  }

  override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
    val taskDescription = TaskDescription.decode(taskInfo.getData.asReadOnlyByteBuffer())
    if (executor == null) {
      logError("Received launchTask but executor was null")
    } else {
      SparkHadoopUtil.get.runAsSparkUser { () =>
        executor.launchTask(this, taskDescription)
      }
    }
  }

  override def error(d: ExecutorDriver, message: String) {
    logError("Error from Mesos: " + message)
  }

  override def killTask(d: ExecutorDriver, t: TaskID) {
    if (executor == null) {
      logError("Received KillTask but executor was null")
    } else {
      // TODO: Determine the 'interruptOnCancel' property set for the given job.
      executor.killTask(t.getValue.toLong, interruptThread = false)
    }
  }

  override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {}

  override def disconnected(d: ExecutorDriver) {}

  override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {}

  override def shutdown(d: ExecutorDriver) {}
}

/**
 * Entry point for Mesos executor.
 */
private[spark] object MesosExecutorBackend extends Logging {
  def main(args: Array[String]) {
    Utils.initDaemon(log)
    // Create a new Executor and start it running
    val runner = new MesosExecutorBackend()
    new MesosExecutorDriver(runner).run()
  }
}