aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
blob: ae56244979270509cfbd1aff4da0cd82a7a8f8d7 (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
package org.apache.spark.scheduler.cluster

/*
 * 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.
 */


import org.apache.spark.{Logging, SparkContext}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, FileSystem}

private[spark] class SimrSchedulerBackend(
                                           scheduler: ClusterScheduler,
                                           sc: SparkContext,
                                           driverFilePath: String)
  extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
  with Logging {

  val tmpPath = new Path(driverFilePath + "_tmp");
  val filePath = new Path(driverFilePath);

  val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt

  override def start() {
    super.start()

    val driverUrl = "akka://spark@%s:%s/user/%s".format(
      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
      CoarseGrainedSchedulerBackend.ACTOR_NAME)

    val conf = new Configuration()
    val fs = FileSystem.get(conf)

    logInfo("Writing to HDFS file: "  + driverFilePath);
    logInfo("Writing AKKA address: "  + driverUrl);

    // Create temporary file to prevent race condition where executors get empty driverUrl file
    val temp = fs.create(tmpPath, true)
    temp.writeUTF(driverUrl)
    temp.writeInt(maxCores)
    temp.close()

    // "Atomic" rename
    fs.rename(tmpPath, filePath);
  }

  override def stop() {
    val conf = new Configuration()
    val fs = FileSystem.get(conf)
    fs.delete(new Path(driverFilePath), false);
    super.stopExecutors()
    super.stop()
  }
}