aboutsummaryrefslogtreecommitdiff
path: root/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
blob: 2e2fb5d4fa787a3be946821de4e9f1a1b46ddbf5 (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
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
/*
 * 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.yarn

import java.nio.ByteBuffer

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.DataOutputBuffer
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.api.protocolrecords._
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, Records}

import org.apache.spark.{Logging, SparkConf}


/**
 * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API.
 */
class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf)
  extends YarnClientImpl with ClientBase with Logging {

  def this(clientArgs: ClientArguments, spConf: SparkConf) =
    this(clientArgs, new Configuration(), spConf)

  def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf())

  val args = clientArgs
  val conf = hadoopConf
  val sparkConf = spConf
  var rpc: YarnRPC = YarnRPC.create(conf)
  val yarnConf: YarnConfiguration = new YarnConfiguration(conf)

  def runApp(): ApplicationId = {
    validateArgs()
    // Initialize and start the client service.
    init(yarnConf)
    start()

    // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers).
    logClusterResourceDetails()

    // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM)
    // interface).

    // Get a new client application.
    val newApp = super.createApplication()
    val newAppResponse = newApp.getNewApplicationResponse()
    val appId = newAppResponse.getApplicationId()

    verifyClusterResources(newAppResponse)

    // Set up resource and environment variables.
    val appStagingDir = getAppStagingDir(appId)
    val localResources = prepareLocalResources(appStagingDir)
    val launchEnv = setupLaunchEnv(localResources, appStagingDir)
    val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv)

    // Set up an application submission context.
    val appContext = newApp.getApplicationSubmissionContext()
    appContext.setApplicationName(args.appName)
    appContext.setQueue(args.amQueue)
    appContext.setAMContainerSpec(amContainer)
    appContext.setApplicationType("SPARK")

    // Memory for the ApplicationMaster.
    val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
    memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
    appContext.setResource(memoryResource)

    // Finally, submit and monitor the application.
    submitApp(appContext)
    appId
  }

  def run() {
    val appId = runApp()
    monitorApplication(appId)
    System.exit(0)
  }

  def logClusterResourceDetails() {
    val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
    logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " +
      clusterMetrics.getNumNodeManagers)

    val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
    logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
      queueApplicationCount = %s, queueChildQueueCount = %s""".format(
        queueInfo.getQueueName,
        queueInfo.getCurrentCapacity,
        queueInfo.getMaximumCapacity,
        queueInfo.getApplications.size,
        queueInfo.getChildQueues.size))
  }

  def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = {
    // TODO: Need a replacement for the following code to fix -Xmx?
    // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
    // var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
    //  ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
    //    YarnAllocationHandler.MEMORY_OVERHEAD)
    args.amMemory
  }

  def setupSecurityToken(amContainer: ContainerLaunchContext) = {
    // Setup security tokens.
    val dob = new DataOutputBuffer()
    credentials.writeTokenStorageToStream(dob)
    amContainer.setTokens(ByteBuffer.wrap(dob.getData()))
  }

  def submitApp(appContext: ApplicationSubmissionContext) = {
    // Submit the application to the applications manager.
    logInfo("Submitting application to ASM")
    super.submitApplication(appContext)
  }

  def monitorApplication(appId: ApplicationId): Boolean = {
    val interval = sparkConf.getLong("spark.yarn.report.interval", 1000)

    while (true) {
      Thread.sleep(interval)
      val report = super.getApplicationReport(appId)

      logInfo("Application report from ASM: \n" +
        "\t application identifier: " + appId.toString() + "\n" +
        "\t appId: " + appId.getId() + "\n" +
        "\t clientToAMToken: " + report.getClientToAMToken() + "\n" +
        "\t appDiagnostics: " + report.getDiagnostics() + "\n" +
        "\t appMasterHost: " + report.getHost() + "\n" +
        "\t appQueue: " + report.getQueue() + "\n" +
        "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
        "\t appStartTime: " + report.getStartTime() + "\n" +
        "\t yarnAppState: " + report.getYarnApplicationState() + "\n" +
        "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" +
        "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
        "\t appUser: " + report.getUser()
      )

      val state = report.getYarnApplicationState()
      if (state == YarnApplicationState.FINISHED ||
        state == YarnApplicationState.FAILED ||
        state == YarnApplicationState.KILLED) {
        return true
      }
    }
    true
  }
}

object Client {

  def main(argStrings: Array[String]) {
    println("WARNING: This client is deprecated and will be removed in a future version of Spark.")
    println("Use ./bin/spark-submit with \"--master yarn\"")

    // Set an env variable indicating we are running in YARN mode.
    // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
    // see Client#setupLaunchEnv().
    System.setProperty("SPARK_YARN_MODE", "true")
    val sparkConf = new SparkConf()
    val args = new ClientArguments(argStrings, sparkConf)

    new Client(args, sparkConf).run()
  }

}