Skip to content

Commit

Permalink
Merge branch 'master' of git://git.apache.org/spark into SPARK-2894
Browse files Browse the repository at this point in the history
  • Loading branch information
sarutak committed Aug 9, 2014
2 parents 2c899ed + b431e67 commit 47f3510
Show file tree
Hide file tree
Showing 28 changed files with 228 additions and 85 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,15 @@

package org.apache.spark.broadcast

import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream}
import java.io.{ByteArrayOutputStream, ByteArrayInputStream, InputStream,
ObjectInputStream, ObjectOutputStream, OutputStream}

import scala.reflect.ClassTag
import scala.util.Random

import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
import org.apache.spark.util.Utils

/**
* A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like
Expand Down Expand Up @@ -214,11 +215,15 @@ private[broadcast] object TorrentBroadcast extends Logging {
private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024
private var initialized = false
private var conf: SparkConf = null
private var compress: Boolean = false
private var compressionCodec: CompressionCodec = null

def initialize(_isDriver: Boolean, conf: SparkConf) {
TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests
synchronized {
if (!initialized) {
compress = conf.getBoolean("spark.broadcast.compress", true)
compressionCodec = CompressionCodec.createCodec(conf)
initialized = true
}
}
Expand All @@ -228,8 +233,13 @@ private[broadcast] object TorrentBroadcast extends Logging {
initialized = false
}

def blockifyObject[T](obj: T): TorrentInfo = {
val byteArray = Utils.serialize[T](obj)
def blockifyObject[T: ClassTag](obj: T): TorrentInfo = {
val bos = new ByteArrayOutputStream()
val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos
val ser = SparkEnv.get.serializer.newInstance()
val serOut = ser.serializeStream(out)
serOut.writeObject[T](obj).close()
val byteArray = bos.toByteArray
val bais = new ByteArrayInputStream(byteArray)

var blockNum = byteArray.length / BLOCK_SIZE
Expand All @@ -255,7 +265,7 @@ private[broadcast] object TorrentBroadcast extends Logging {
info
}

def unBlockifyObject[T](
def unBlockifyObject[T: ClassTag](
arrayOfBlocks: Array[TorrentBlock],
totalBytes: Int,
totalBlocks: Int): T = {
Expand All @@ -264,7 +274,16 @@ private[broadcast] object TorrentBroadcast extends Logging {
System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray,
i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length)
}
Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader)

val in: InputStream = {
val arrIn = new ByteArrayInputStream(retByteArray)
if (compress) compressionCodec.compressedInputStream(arrIn) else arrIn
}
val ser = SparkEnv.get.serializer.newInstance()
val serIn = ser.deserializeStream(in)
val obj = serIn.readObject[T]()
serIn.close()
obj
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ private[spark] class Worker(
logInfo("Spark home: " + sparkHome)
createWorkDir()
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
webUi = new WorkerWebUI(this, workDir, webUiPort)
webUi.bind()
registerWithMaster()

Expand Down Expand Up @@ -373,7 +373,8 @@ private[spark] class Worker(
private[spark] object Worker extends Logging {
def main(argStrings: Array[String]) {
SignalLogger.register(log)
val args = new WorkerArguments(argStrings)
val conf = new SparkConf
val args = new WorkerArguments(argStrings, conf)
val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
args.memory, args.masters, args.workDir)
actorSystem.awaitTermination()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,12 @@ package org.apache.spark.deploy.worker
import java.lang.management.ManagementFactory

import org.apache.spark.util.{IntParam, MemoryParam, Utils}
import org.apache.spark.SparkConf

/**
* Command-line parser for the worker.
*/
private[spark] class WorkerArguments(args: Array[String]) {
private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
var host = Utils.localHostName()
var port = 0
var webUiPort = 8081
Expand All @@ -46,6 +47,9 @@ private[spark] class WorkerArguments(args: Array[String]) {
if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) {
webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt
}
if (conf.contains("spark.worker.ui.port")) {
webUiPort = conf.get("spark.worker.ui.port").toInt
}
if (System.getenv("SPARK_WORKER_DIR") != null) {
workDir = System.getenv("SPARK_WORKER_DIR")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@ private[spark]
class WorkerWebUI(
val worker: Worker,
val workDir: File,
port: Option[Int] = None)
extends WebUI(worker.securityMgr, getUIPort(port, worker.conf), worker.conf, name = "WorkerUI")
requestedPort: Int)
extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI")
with Logging {

val timeout = AkkaUtils.askTimeout(worker.conf)
Expand All @@ -55,10 +55,5 @@ class WorkerWebUI(
}

private[spark] object WorkerWebUI {
val DEFAULT_PORT = 8081
val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR

def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = {
requestedPort.getOrElse(conf.getInt("spark.worker.ui.port", WorkerWebUI.DEFAULT_PORT))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -119,11 +119,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {

/**
* Compute a histogram using the provided buckets. The buckets are all open
* to the left except for the last which is closed
* to the right except for the last which is closed
* e.g. for the array
* [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50]
* e.g 1<=x<10 , 10<=x<20, 20<=x<50
* And on the input of 1 and 50 we would have a histogram of 1, 0, 0
* e.g 1<=x<10 , 10<=x<20, 20<=x<=50
* And on the input of 1 and 50 we would have a histogram of 1, 0, 1
*
* Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
* from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1233,6 +1233,11 @@ abstract class RDD[T: ClassTag](
dependencies.head.rdd.asInstanceOf[RDD[U]]
}

/** Returns the jth parent RDD: e.g. rdd.parent[T](0) is equivalent to rdd.firstParent[T] */
protected[spark] def parent[U: ClassTag](j: Int) = {
dependencies(j).rdd.asInstanceOf[RDD[U]]
}

/** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
def context = sc

Expand Down
3 changes: 1 addition & 2 deletions core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,7 @@ class UnionRDD[T: ClassTag](

override def compute(s: Partition, context: TaskContext): Iterator[T] = {
val part = s.asInstanceOf[UnionPartition[T]]
val parentRdd = dependencies(part.parentRddIndex).rdd.asInstanceOf[RDD[T]]
parentRdd.iterator(part.parentPartition, context)
parent[T](part.parentRddIndex).iterator(part.parentPartition, context)
}

override def getPreferredLocations(s: Partition): Seq[String] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,19 +47,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
{
// Use an atomic variable to track total number of cores in the cluster for simplicity and speed
var totalCoreCount = new AtomicInteger(0)
var totalExpectedExecutors = new AtomicInteger(0)
var totalRegisteredExecutors = new AtomicInteger(0)
val conf = scheduler.sc.conf
private val timeout = AkkaUtils.askTimeout(conf)
private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf)
// Submit tasks only after (registered executors / total expected executors)
// Submit tasks only after (registered resources / total expected resources)
// is equal to at least this value, that is double between 0 and 1.
var minRegisteredRatio = conf.getDouble("spark.scheduler.minRegisteredExecutorsRatio", 0)
if (minRegisteredRatio > 1) minRegisteredRatio = 1
// Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the time(milliseconds).
var minRegisteredRatio =
math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0))
// Submit tasks after maxRegisteredWaitingTime milliseconds
// if minRegisteredRatio has not yet been reached
val maxRegisteredWaitingTime =
conf.getInt("spark.scheduler.maxRegisteredExecutorsWaitingTime", 30000)
conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000)
val createTime = System.currentTimeMillis()
var ready = if (minRegisteredRatio <= 0) true else false

class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
private val executorActor = new HashMap[String, ActorRef]
Expand Down Expand Up @@ -94,12 +94,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
executorAddress(executorId) = sender.path.address
addressToExecutorId(sender.path.address) = executorId
totalCoreCount.addAndGet(cores)
if (executorActor.size >= totalExpectedExecutors.get() * minRegisteredRatio && !ready) {
ready = true
logInfo("SchedulerBackend is ready for scheduling beginning, registered executors: " +
executorActor.size + ", total expected executors: " + totalExpectedExecutors.get() +
", minRegisteredExecutorsRatio: " + minRegisteredRatio)
}
totalRegisteredExecutors.addAndGet(1)
makeOffers()
}

Expand Down Expand Up @@ -268,14 +263,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
}
}

def sufficientResourcesRegistered(): Boolean = true

override def isReady(): Boolean = {
if (ready) {
if (sufficientResourcesRegistered) {
logInfo("SchedulerBackend is ready for scheduling beginning after " +
s"reached minRegisteredResourcesRatio: $minRegisteredRatio")
return true
}
if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) {
ready = true
logInfo("SchedulerBackend is ready for scheduling beginning after waiting " +
"maxRegisteredExecutorsWaitingTime: " + maxRegisteredWaitingTime)
s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)")
return true
}
false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ private[spark] class SparkDeploySchedulerBackend(
var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _

val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
val totalExpectedCores = maxCores.getOrElse(0)

override def start() {
super.start()
Expand Down Expand Up @@ -97,7 +98,6 @@ private[spark] class SparkDeploySchedulerBackend(

override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int,
memory: Int) {
totalExpectedExecutors.addAndGet(1)
logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format(
fullId, hostPort, cores, Utils.megabytesToString(memory)))
}
Expand All @@ -110,4 +110,8 @@ private[spark] class SparkDeploySchedulerBackend(
logInfo("Executor %s removed: %s".format(fullId, message))
removeExecutor(fullId.split("/")(1), reason.toString)
}

override def sufficientResourcesRegistered(): Boolean = {
totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {

test("Accessing HttpBroadcast variables in a local cluster") {
val numSlaves = 4
sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf)
val conf = httpConf.clone
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
conf.set("spark.broadcast.compress", "true")
sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf)
val list = List[Int](1, 2, 3, 4)
val broadcast = sc.broadcast(list)
val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum))
Expand All @@ -69,7 +72,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext {

test("Accessing TorrentBroadcast variables in a local cluster") {
val numSlaves = 4
sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf)
val conf = torrentConf.clone
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
conf.set("spark.broadcast.compress", "true")
sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf)
val list = List[Int](1, 2, 3, 4)
val broadcast = sc.broadcast(list)
val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum))
Expand Down
10 changes: 10 additions & 0 deletions core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -726,6 +726,16 @@ class RDDSuite extends FunSuite with SharedSparkContext {
jrdd.rdd.retag.collect()
}

test("parent method") {
val rdd1 = sc.parallelize(1 to 10, 2)
val rdd2 = rdd1.filter(_ % 2 == 0)
val rdd3 = rdd2.map(_ + 1)
val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3))
assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]])
assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]])
assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]])
}

test("getNarrowAncestors") {
val rdd1 = sc.parallelize(1 to 100, 4)
val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1)
Expand Down
13 changes: 7 additions & 6 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -825,21 +825,22 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
</tr>
<td><code>spark.scheduler.minRegisteredExecutorsRatio</code></td>
<td><code>spark.scheduler.minRegisteredResourcesRatio</code></td>
<td>0</td>
<td>
The minimum ratio of registered executors (registered executors / total expected executors)
The minimum ratio of registered resources (registered resources / total expected resources)
(resources are executors in yarn mode, CPU cores in standalone mode)
to wait for before scheduling begins. Specified as a double between 0 and 1.
Regardless of whether the minimum ratio of executors has been reached,
Regardless of whether the minimum ratio of resources has been reached,
the maximum amount of time it will wait before scheduling begins is controlled by config
<code>spark.scheduler.maxRegisteredExecutorsWaitingTime</code>
<code>spark.scheduler.maxRegisteredResourcesWaitingTime</code>
</td>
</tr>
<tr>
<td><code>spark.scheduler.maxRegisteredExecutorsWaitingTime</code></td>
<td><code>spark.scheduler.maxRegisteredResourcesWaitingTime</code></td>
<td>30000</td>
<td>
Maximum amount of time to wait for executors to register before scheduling begins
Maximum amount of time to wait for resources to register before scheduling begins
(in milliseconds).
</td>
</tr>
Expand Down
2 changes: 1 addition & 1 deletion mllib/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@
<dependency>
<groupId>org.scalanlp</groupId>
<artifactId>breeze_${scala.binary.version}</artifactId>
<version>0.7</version>
<version>0.9</version>
<exclusions>
<!-- This is included as a compile-scoped dependency by jtransforms, which is
a dependency of breeze. -->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,7 @@ class RowMatrix(
EigenValueDecomposition.symmetricEigs(v => G * v, n, k, tol, maxIter)
case SVDMode.LocalLAPACK =>
val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]]
val (uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G)
val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G)
(sigmaSquaresFull, uFull)
case SVDMode.DistARPACK =>
require(k < n, s"k must be smaller than n in dist-eigs mode but got k=$k and n=$n.")
Expand Down Expand Up @@ -338,7 +338,7 @@ class RowMatrix(

val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]]

val (u: BDM[Double], _, _) = brzSvd(Cov)
val brzSvd.SVD(u: BDM[Double], _, _) = brzSvd(Cov)

if (k == n) {
Matrices.dense(n, k, u.data)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext {
for (mat <- Seq(denseMat, sparseMat)) {
for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) {
val localMat = mat.toBreeze()
val (localU, localSigma, localVt) = brzSvd(localMat)
val brzSvd.SVD(localU, localSigma, localVt) = brzSvd(localMat)
val localV: BDM[Double] = localVt.t.toDenseMatrix
for (k <- 1 to n) {
val skip = (mode == "local-eigs" || mode == "dist-eigs") && k == n
Expand Down
4 changes: 4 additions & 0 deletions project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,10 @@ object MimaExcludes {
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$")
) ++
Seq ( // package-private classes removed in MLlib
ProblemFilters.exclude[MissingMethodProblem](
"org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne")
)
case v if v.startsWith("1.0") =>
Seq(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,9 @@ private[sql] object JsonRDD extends Logging {
val newType = dataType match {
case NullType => StringType
case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull)
case struct: StructType => nullTypeToStringType(struct)
case ArrayType(struct: StructType, containsNull) =>
ArrayType(nullTypeToStringType(struct), containsNull)
case struct: StructType =>nullTypeToStringType(struct)
case other: DataType => other
}
StructField(fieldName, newType, nullable)
Expand Down

0 comments on commit 47f3510

Please sign in to comment.