Spark源码——Spark on YARN Executor执行Task的过程

我们知道Spark中的Task是由Executor进程中执行的,那么Executor启动之后,具体是如何执行Task的呢?

Executor执行Task的入口为object CoarseGrainedExecutorBackend。CoarseGrainedExecutorBackend是一个管理单个Executor的ExecutorBackend,它是在分配好的Container中其启动Executor后,通过命令行执行main()来启动的。

为了更清楚地分析Executor执行Task的整个流程,我们先看下这部分的执行流程图:
在这里插入图片描述
接下来,我会结合上面的流程图,一步步的来分析整个Task的执行流程。

入口点

在分析 CoarseGrainedExecutorBackend 之前,我们先要了解 YarnAllocator 类,此类的作用主要就是负责向 YARN ResourceManager 请求运行Executor的资源(也就是containers),并且决定在YARN满足这些请求之后,如何处理这些containers。因为containers的启动过程比较复杂,之后我会单独分析这块的源码。

我们先看 YarnAllocator 类的 runAllocatedContainers() 方法,此方法就是负责在分配的containers中启动executors:

private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = {
  for (container <- containersToUse) {
    ...
    if (runningExecutors.size() < targetNumExecutors) {
      numExecutorsStarting.incrementAndGet()
      if (launchContainers) {
      	// launcherPool是一个cached线程池,表示YARN ApplicationMaster用来启动executor container的最大线程数
        launcherPool.execute(new Runnable {
          override def run(): Unit = {
            try {
              new ExecutorRunnable(
                Some(container),
                conf,
                sparkConf,
                driverUrl,
                executorId,
                executorHostname,
                executorMemory,
                executorCores,
                appAttemptId.getApplicationId.toString,
                securityMgr,
                localResources
              ).run()
              updateInternalState()
            } catch {
              ...
            }
          }
        })
      } else {
        // For test only
        updateInternalState()
      }
    } else {
      logInfo(("Skip launching executorRunnable as running executors count: %d " +
        "reached target executors count: %d.").format(runningExecutors.size, targetNumExecutors))
    }
  }
}

上面代码中的 execute() 方法会从线程池中获取一个线程来创建一个 ExecutorRunnable 实例,并在 ExecutorRunnable 类的 run() 方法中启动一个container:

def run(): Unit = {
  logDebug("Starting Executor Container")
  nmClient = NMClient.createNMClient()
  nmClient.init(conf)
  nmClient.start()
  startContainer()
}

startContainer() 方法主要负责准备container启动环境、准备启动 CoarseGrainedExecutorBackend 的命令以及启动container。

  def startContainer(): java.util.Map[String, ByteBuffer] = {
    val ctx = Records.newRecord(classOf[ContainerLaunchContext]).asInstanceOf[ContainerLaunchContext]
    val env = prepareEnvironment().asJava
    ...
    val commands = prepareCommand()

    // 是否启用外部Shuffle服务
    if (sparkConf.get(SHUFFLE_SERVICE_ENABLED)) {
      ...
    }

    // 向ContainerManager发送启动container的请求
    try {
      nmClient.startContainer(container.get, ctx)
    } catch {
      case ex: Exception =>
        throw new SparkException(s"Exception while starting container ${container.get.getId}" + s" on host $hostname", ex)
    }
  }

prepareCommand() 函数就是生成启动 CoarseGrainedExecutorBackend 的命令的,通过命令行向 CoarseGrainedExecutorBackend 类的main()函数传参数,比如driver url、启动executor的主机名、executorMemory(会作为-Xmx的参数)、executorCores等重要参数:

private def prepareCommand(): List[String] = {
  ...
  // 设置JVM内存
  val executorMemoryString = executorMemory + "m"
  javaOpts += "-Xmx" + executorMemoryString
  ... 
  val commands = prefixEnv ++
    Seq(Environment.JAVA_HOME.$$() + "/bin/java", "-server") ++
    javaOpts ++
    Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend",
      "--driver-url", masterAddress,
      "--executor-id", executorId,
      "--hostname", hostname,
      "--cores", executorCores.toString,
      "--app-id", appId) ++
    userClassPath ++
    Seq(
      s"1>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stdout",
      s"2>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stderr")
  ...
}

main()、run()

object CoarseGrainedExecutorBackend中的main() 函数主要做一些参数有效性的检查,并调用run()方法:

private def run(
    driverUrl: String,
    executorId: String,
    hostname: String,
    cores: Int,
    appId: String,
    workerUrl: Option[String],
    userClassPath: Seq[URL]) {
  Utils.initDaemon(log)
  //作为Spark用户在安全的环境中运行
  SparkHadoopUtil.get.runAsSparkUser { () =>
    // 主机名检查,主机名不能为空且不能出现“:”
    Utils.checkHost(hostname)

    // 从driver获取Spark配置
    val executorConf = new SparkConf
    //创建一个RpcEnv,用来向driver抓取Spark配置
    val fetcher = RpcEnv.create(
      "driverPropsFetcher",
      hostname,
      -1,
      executorConf,
      new SecurityManager(executorConf),
      clientMode = true)
    //通过RpcEnv创建driver RpcEndpointRef
    val driver: RpcEndpointRef = fetcher.setupEndpointRefByURI(driverUrl)
    //向driver发送消息,获取SparkAppConfig
    val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig)
    //从SparkAppConfig中获取Spark配置, 并添加Spark应用ID
    val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId))
    //关闭RpcEnv
    fetcher.shutdown()
    
    // 使用从driver获取的Spark配置来为executor创建SparkEnv
    val driverConf = new SparkConf()
    for ((key, value) <- props) {
      // this is required for SSL in standalone mode
      if (SparkConf.isExecutorStartupConf(key)) {
        driverConf.setIfMissing(key, value)
      } else {
        driverConf.set(key, value)
      }
    }
    cfg.hadoopDelegationCreds.foreach { tokens =>
      SparkHadoopUtil.get.addDelegationTokens(tokens, driverConf)
    }
    //为executor创建SparkEnv
    val env = SparkEnv.createExecutorEnv(
      driverConf, executorId, hostname, cores, cfg.ioEncryptionKey, isLocal = false)

	//使用名字“Executor”和CoarseGrainedExecutorBackend实例注册一个RpcEndpoint
    env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend(
      env.rpcEnv, driverUrl, executorId, hostname, cores, userClassPath, env))
    //命令行参数中存在--worker-url的情况,仅在standalone模式下使用
    workerUrl.foreach { url =>
      env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url))
    }
    //run()线程会一直阻塞直到RpcEnv中断退出
    env.rpcEnv.awaitTermination()
  }
}

run() 函数实现的功能:

  1. 从driver拉取Spark配置信息,并用来为executor创建SparkEnv环境,
  2. 使用CoarseGrainedExecutorBackend实例注册RpcEndpoint,
  3. 分发消息并处理消息。

env.rpcEnv.setupEndpoint()

setupEndpoint()函数使用名字“Executor”和CoarseGrainedExecutorBackend实例向rpc环境注册一个RpcEndpoint。

private[spark] abstract class RpcEnv(conf: SparkConf) {
	def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef
}
//NettyRpcEnv为RpcEnv的子类
private[netty] class NettyRpcEnv(
    val conf: SparkConf,
    javaSerializerInstance: JavaSerializerInstance,
    host: String,
    securityManager: SecurityManager,
    numUsableCores: Int) extends RpcEnv(conf) with Logging {
	override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = {
	  //由Dispatcher负责注册RpcEndpoint
	  dispatcher.registerRpcEndpoint(name, endpoint)
	}
}
//Dispatcher是一个消息分配器,负责将RPC消息分配到合适的Endpoint。
private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) extends Logging {
	def registerRpcEndpoint(name: String, endpoint: RpcEndpoint): NettyRpcEndpointRef = {
	  //RPC Endpoint的地址标识
	  //nettyEnv.address:Endpoint的socket地址
	  //name:Endpoint名字
	  val addr = RpcEndpointAddress(nettyEnv.address, name)
	  //NettyRpcEndpointRef是RpcEndpointRef的netty版本,是一个远程RpcEndpoint的引用
	  val endpointRef = new NettyRpcEndpointRef(nettyEnv.conf, addr, nettyEnv)
	  synchronized {
	    if (stopped) {
	      throw new IllegalStateException("RpcEnv has been stopped")
	    }
	    if (endpoints.putIfAbsent(name, new EndpointData(name, endpoint, endpointRef)) != null) {
	      throw new IllegalArgumentException(s"There is already an RpcEndpoint called $name")
	    }
	    val data = endpoints.get(name)
	    endpointRefs.put(data.endpoint, data.ref)
	    //保存Endpoint消息,等待后序处理
	    receivers.offer(data)
	  }
	  endpointRef
	}
}

receivers

env.rpcEnv.awaitTermination()

awaitTermination()函数等待RpcEnv处理之前保存到消息队列中的消息,直到rpcEnv处理完消息之后退出:

private[spark] abstract class RpcEnv(conf: SparkConf) {
  def awaitTermination(): Unit
}

private[netty] class NettyRpcEnv(...) extends RpcEnv(conf) with Logging {
  override def awaitTermination(): Unit = {
    dispatcher.awaitTermination()
  }
}

private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) extends Logging {
  def awaitTermination(): Unit = {
    threadpool.awaitTermination(Long.MaxValue, TimeUnit.MILLISECONDS)
  }
//用来分派消息的线程池
private val threadpool: ThreadPoolExecutor = {
    val availableCores =
      if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors()
    val numThreads = nettyEnv.conf.getInt("spark.rpc.netty.dispatcher.numThreads",
      math.max(2, availableCores))
    val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop")
    for (i <- 0 until numThreads) {
      pool.execute(new MessageLoop)
    }
    pool
  }
}
//用来发送消息的消息循环线程
private class MessageLoop extends Runnable {
    override def run(): Unit = {
      try {
        while (true) {
          try {
            //从上一步保存Endpoint消息的队列中取出一条消息
            val data = receivers.take()
            ...
            //处理取出的消息
            data.inbox.process(Dispatcher.this)
          } catch {
            case NonFatal(e) => logError(e.getMessage, e)
          }
        }
      } catch {
        ...
      }
    }
  }

Inbox.process()

不同类型消息的处理是通过 Inbox 类的 process() 函数来完成的,比如OnStart消息、OnStop消息、RpcMessage、OneWayMessage消息等等。

//Inbox用来存储 Rpcendpoint 消息,并且向inbox中保存消息是线程安全的
private[netty] class Inbox(
    val endpointRef: NettyRpcEndpointRef,
    val endpoint: RpcEndpoint)
  extends Logging {
	// OnStart 消息会首先被处理
	inbox.synchronized {
	  messages.add(OnStart)
	}
	//保存各种类型的消息
	protected val messages = new java.util.LinkedList[InboxMessage]()
	
	def process(dispatcher: Dispatcher): Unit = {
	  var message: InboxMessage = null
	  inbox.synchronized {
	    ...
	    message = messages.poll()
	    ...
	  }
	  while (true) {
	    safelyCall(endpoint) {
	      message match {
	        case RpcMessage(_sender, content, context) =>
	          try {
	            endpoint.receiveAndReply(context).applyOrElse[Any, Unit](content, { msg =>
	              throw new SparkException(s"Unsupported message $message from ${_sender}")
	            })
	          } catch {
	            ...
	          }
			//此种类型消息的处理会转发到CoarseGrainedExecutorBackend类的receive函数来处理
	        case OneWayMessage(_sender, content) =>
	          endpoint.receive.applyOrElse[Any, Unit](content, { msg =>
	            throw new SparkException(s"Unsupported message $message from ${_sender}")
	          })
	        case OnStart =>
	          //必须在RpcEndpoint开始处理消息之前调用
	          endpoint.onStart()
	          ...
	        case OnStop =>
	          ...
	          dispatcher.removeRpcEndpointRef(endpoint)
	          endpoint.onStop()
	          assert(isEmpty, "OnStop should be the last message")
	
	        ...
	      }
	    }
	    ...
	  }
	}
}

上面代码中的onStart()函数的真正实现是CoarseGrainedExecutorBackend类的onStart()函数:

  override def onStart() {
    logInfo("Connecting to driver: " + driverUrl)
    //异步地获取指定uri的RpcEndpointRef
    rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref =>
      // This is a very fast action so we can use "ThreadUtils.sameThread"
      driver = Some(ref)
      //driver向executor发送消息,并让executor向自己注册
      ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls))
    }(ThreadUtils.sameThread).onComplete {
      // This is a very fast action so we can use "ThreadUtils.sameThread"
      case Success(msg) =>
        // Always receive `true`. Just ignore it
      case Failure(e) =>
        exitExecutor(1, s"Cannot register with driver: $driverUrl", e, notifyDriver = false)
    }(ThreadUtils.sameThread)
  }

Task启动消息

Inbox类 process() 函数处理 OneWayMessage 消息的实现会转发到CoarseGrainedExecutorBackend类的receive函数中:

  override def receive: PartialFunction[Any, Unit] = {
    //executor向driver进行注册
    case RegisteredExecutor =>
      logInfo("Successfully registered with driver")
      try {
        //创建Executor实例
        executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false)
      } catch {
        case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e)
      }

    case RegisterExecutorFailed(message) =>
      exitExecutor(1, "Slave registration failed: " + message)

    //启动task消息
    case LaunchTask(data) =>
      if (executor == null) {
        exitExecutor(1, "Received LaunchTask command but executor was null")
      } else {
        //task描述
        val taskDesc = TaskDescription.decode(data.value)
        logInfo("Got assigned task " + taskDesc.taskId)
        //启动task
        executor.launchTask(this, taskDesc)
      }
      
    case KillTask(taskId, _, interruptThread, reason) =>
      if (executor == null) {
        exitExecutor(1, "Received KillTask command but executor was null")
      } else {
        executor.killTask(taskId, interruptThread, reason)
      }
   ...
  }
  • LaunchTask(data):首先对数据进行解码生成TaskDescription,然后在指定executor上启动task。

LaunchTask消息是在 DriverEndpoint类的 launchTasks() 函数中请求启动task的时候被发送的:

    // Launch tasks returned by a set of resource offers
    private def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
      for (task <- tasks.flatten) {
        //序列化task,返回一个ByteBuffer
        val serializedTask: ByteBuffer = TaskDescription.encode(task)
        //被序列化task的大小不能超过最大的rpc消息的大小,否则任务被中断
        if (serializedTask.limit() >= maxRpcMessageSize) {
          Option(scheduler.taskIdToTaskSetManager.get(task.taskId)).foreach { taskSetMgr =>
            try {
              var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " +
                "spark.rpc.message.maxSize (%d bytes). Consider increasing " +
                "spark.rpc.message.maxSize or using broadcast variables for large values."
              msg = msg.format(task.taskId, task.index, serializedTask.limit(), maxRpcMessageSize)
              taskSetMgr.abort(msg)
            } catch {
              case e: Exception => logError("Exception in error callback", e)
            }
          }
        } else {
          val executorData = executorDataMap(task.executorId)
          //启动一个task,对应的executor上CPU减1,默认启动一个task使用一个CPU core
          executorData.freeCores -= scheduler.CPUS_PER_TASK

          logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " +
            s"${executorData.executorHost}.")
		  //向NettyRpcEndpointRef发送启动task的消息
          executorData.executorEndpoint.send(LaunchTask(new SerializableBuffer(serializedTask)))
        }
      }
    }

Task的执行-launchTask()

最终task的执行是由 Executor 类的 launchTask() 方法来完成的:

// Spark executor配合线程池来运行task
private[spark] class Executor(
    executorId: String,
    executorHostname: String,
    env: SparkEnv,
    userClassPath: Seq[URL] = Nil,
    isLocal: Boolean = false,
    uncaughtExceptionHandler: UncaughtExceptionHandler = new SparkUncaughtExceptionHandler)
  extends Logging {
  //维护正在运行的task
  private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]

  //启动task
  def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = {
    //实例化一个线程,TaskRunner是继承之Runnable接口的
    val tr = new TaskRunner(context, taskDescription)
    runningTasks.put(taskDescription.taskId, tr)
    //线程池执行task
    threadPool.execute(tr)
  }
}
  • 0
    点赞
  • 0
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值