Skip to content

Commit a2eb3b9

Browse files
committed
Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller
2 parents 475e346 + 15724fa commit a2eb3b9

File tree

279 files changed

+7910
-5468
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

279 files changed

+7910
-5468
lines changed

R/pkg/R/DataFrame.R

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1566,7 +1566,7 @@ setMethod("fillna",
15661566
#' @return a local R data.frame representing the contingency table. The first column of each row
15671567
#' will be the distinct values of `col1` and the column names will be the distinct values
15681568
#' of `col2`. The name of the first column will be `$col1_$col2`. Pairs that have no
1569-
#' occurrences will have `null` as their counts.
1569+
#' occurrences will have zero as their counts.
15701570
#'
15711571
#' @rdname statfunctions
15721572
#' @export

R/pkg/R/deserialize.R

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -102,11 +102,11 @@ readList <- function(con) {
102102

103103
readRaw <- function(con) {
104104
dataLen <- readInt(con)
105-
data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
105+
readBin(con, raw(), as.integer(dataLen), endian = "big")
106106
}
107107

108108
readRawLen <- function(con, dataLen) {
109-
data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
109+
readBin(con, raw(), as.integer(dataLen), endian = "big")
110110
}
111111

112112
readDeserialize <- function(con) {

R/pkg/R/sparkR.R

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -104,16 +104,13 @@ sparkR.init <- function(
104104
return(get(".sparkRjsc", envir = .sparkREnv))
105105
}
106106

107-
sparkMem <- Sys.getenv("SPARK_MEM", "1024m")
108107
jars <- suppressWarnings(normalizePath(as.character(sparkJars)))
109108

110109
# Classpath separator is ";" on Windows
111110
# URI needs four /// as from http://stackoverflow.com/a/18522792
112111
if (.Platform$OS.type == "unix") {
113-
collapseChar <- ":"
114112
uriSep <- "//"
115113
} else {
116-
collapseChar <- ";"
117114
uriSep <- "////"
118115
}
119116

R/pkg/inst/tests/test_mllib.R

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -35,8 +35,8 @@ test_that("glm and predict", {
3535

3636
test_that("predictions match with native glm", {
3737
training <- createDataFrame(sqlContext, iris)
38-
model <- glm(Sepal_Width ~ Sepal_Length, data = training)
38+
model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training)
3939
vals <- collect(select(predict(model, training), "prediction"))
40-
rVals <- predict(glm(Sepal.Width ~ Sepal.Length, data = iris), iris)
41-
expect_true(all(abs(rVals - vals) < 1e-9), rVals - vals)
40+
rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris)
41+
expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals)
4242
})

bin/pyspark

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,4 +82,4 @@ fi
8282

8383
export PYSPARK_DRIVER_PYTHON
8484
export PYSPARK_DRIVER_PYTHON_OPTS
85-
exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main "$@"
85+
exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main --name "PySparkShell" "$@"

bin/pyspark2.cmd

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,4 +35,4 @@ set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH%
3535
set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
3636
set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py
3737

38-
call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main %*
38+
call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main --name "PySparkShell" %*

conf/log4j.properties.template

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
1010
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
1111
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
1212
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
13+
14+
# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
15+
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
16+
log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -150,6 +150,11 @@ private long getMemoryUsage() {
150150
return sorter.getMemoryUsage() + (allocatedPages.size() * (long) PAGE_SIZE);
151151
}
152152

153+
@VisibleForTesting
154+
public int getNumberOfAllocatedPages() {
155+
return allocatedPages.size();
156+
}
157+
153158
public long freeMemory() {
154159
long memoryFreed = 0;
155160
for (MemoryBlock block : allocatedPages) {
@@ -257,7 +262,7 @@ public void insertRecord(
257262
currentPagePosition,
258263
lengthInBytes);
259264
currentPagePosition += lengthInBytes;
260-
265+
freeSpaceInCurrentPage -= totalSpaceRequired;
261266
sorter.insertRecord(recordAddress, prefix);
262267
}
263268

core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
1010
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
1111
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
1212
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
13+
14+
# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
15+
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
16+
log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

core/src/main/resources/org/apache/spark/log4j-defaults.properties

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
1010
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
1111
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
1212
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
13+
14+
# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
15+
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
16+
log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR

core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -24,11 +24,23 @@ package org.apache.spark
2424
private[spark] trait ExecutorAllocationClient {
2525

2626
/**
27-
* Express a preference to the cluster manager for a given total number of executors.
28-
* This can result in canceling pending requests or filing additional requests.
27+
* Update the cluster manager on our scheduling needs. Three bits of information are included
28+
* to help it make decisions.
29+
* @param numExecutors The total number of executors we'd like to have. The cluster manager
30+
* shouldn't kill any running executor to reach this number, but,
31+
* if all existing executors were to die, this is the number of executors
32+
* we'd want to be allocated.
33+
* @param localityAwareTasks The number of tasks in all active stages that have a locality
34+
* preferences. This includes running, pending, and completed tasks.
35+
* @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
36+
* that would like to like to run on that host.
37+
* This includes running, pending, and completed tasks.
2938
* @return whether the request is acknowledged by the cluster manager.
3039
*/
31-
private[spark] def requestTotalExecutors(numExecutors: Int): Boolean
40+
private[spark] def requestTotalExecutors(
41+
numExecutors: Int,
42+
localityAwareTasks: Int,
43+
hostToLocalTaskCount: Map[String, Int]): Boolean
3244

3345
/**
3446
* Request an additional number of executors from the cluster manager.

core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala

Lines changed: 60 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -161,6 +161,12 @@ private[spark] class ExecutorAllocationManager(
161161
// (2) an executor idle timeout has elapsed.
162162
@volatile private var initializing: Boolean = true
163163

164+
// Number of locality aware tasks, used for executor placement.
165+
private var localityAwareTasks = 0
166+
167+
// Host to possible task running on it, used for executor placement.
168+
private var hostToLocalTaskCount: Map[String, Int] = Map.empty
169+
164170
/**
165171
* Verify that the settings specified through the config are valid.
166172
* If not, throw an appropriate exception.
@@ -295,7 +301,7 @@ private[spark] class ExecutorAllocationManager(
295301

296302
// If the new target has not changed, avoid sending a message to the cluster manager
297303
if (numExecutorsTarget < oldNumExecutorsTarget) {
298-
client.requestTotalExecutors(numExecutorsTarget)
304+
client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
299305
logDebug(s"Lowering target number of executors to $numExecutorsTarget (previously " +
300306
s"$oldNumExecutorsTarget) because not all requested executors are actually needed")
301307
}
@@ -349,7 +355,8 @@ private[spark] class ExecutorAllocationManager(
349355
return 0
350356
}
351357

352-
val addRequestAcknowledged = testing || client.requestTotalExecutors(numExecutorsTarget)
358+
val addRequestAcknowledged = testing ||
359+
client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
353360
if (addRequestAcknowledged) {
354361
val executorsString = "executor" + { if (delta > 1) "s" else "" }
355362
logInfo(s"Requesting $delta new $executorsString because tasks are backlogged" +
@@ -519,13 +526,37 @@ private[spark] class ExecutorAllocationManager(
519526
// Number of tasks currently running on the cluster. Should be 0 when no stages are active.
520527
private var numRunningTasks: Int = _
521528

529+
// stageId to tuple (the number of task with locality preferences, a map where each pair is a
530+
// node and the number of tasks that would like to be scheduled on that node) map,
531+
// maintain the executor placement hints for each stage Id used by resource framework to better
532+
// place the executors.
533+
private val stageIdToExecutorPlacementHints = new mutable.HashMap[Int, (Int, Map[String, Int])]
534+
522535
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
523536
initializing = false
524537
val stageId = stageSubmitted.stageInfo.stageId
525538
val numTasks = stageSubmitted.stageInfo.numTasks
526539
allocationManager.synchronized {
527540
stageIdToNumTasks(stageId) = numTasks
528541
allocationManager.onSchedulerBacklogged()
542+
543+
// Compute the number of tasks requested by the stage on each host
544+
var numTasksPending = 0
545+
val hostToLocalTaskCountPerStage = new mutable.HashMap[String, Int]()
546+
stageSubmitted.stageInfo.taskLocalityPreferences.foreach { locality =>
547+
if (!locality.isEmpty) {
548+
numTasksPending += 1
549+
locality.foreach { location =>
550+
val count = hostToLocalTaskCountPerStage.getOrElse(location.host, 0) + 1
551+
hostToLocalTaskCountPerStage(location.host) = count
552+
}
553+
}
554+
}
555+
stageIdToExecutorPlacementHints.put(stageId,
556+
(numTasksPending, hostToLocalTaskCountPerStage.toMap))
557+
558+
// Update the executor placement hints
559+
updateExecutorPlacementHints()
529560
}
530561
}
531562

@@ -534,6 +565,10 @@ private[spark] class ExecutorAllocationManager(
534565
allocationManager.synchronized {
535566
stageIdToNumTasks -= stageId
536567
stageIdToTaskIndices -= stageId
568+
stageIdToExecutorPlacementHints -= stageId
569+
570+
// Update the executor placement hints
571+
updateExecutorPlacementHints()
537572

538573
// If this is the last stage with pending tasks, mark the scheduler queue as empty
539574
// This is needed in case the stage is aborted for any reason
@@ -637,6 +672,29 @@ private[spark] class ExecutorAllocationManager(
637672
def isExecutorIdle(executorId: String): Boolean = {
638673
!executorIdToTaskIds.contains(executorId)
639674
}
675+
676+
/**
677+
* Update the Executor placement hints (the number of tasks with locality preferences,
678+
* a map where each pair is a node and the number of tasks that would like to be scheduled
679+
* on that node).
680+
*
681+
* These hints are updated when stages arrive and complete, so are not up-to-date at task
682+
* granularity within stages.
683+
*/
684+
def updateExecutorPlacementHints(): Unit = {
685+
var localityAwareTasks = 0
686+
val localityToCount = new mutable.HashMap[String, Int]()
687+
stageIdToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) =>
688+
localityAwareTasks += numTasksPending
689+
localities.foreach { case (hostname, count) =>
690+
val updatedCount = localityToCount.getOrElse(hostname, 0) + count
691+
localityToCount(hostname) = updatedCount
692+
}
693+
}
694+
695+
allocationManager.localityAwareTasks = localityAwareTasks
696+
allocationManager.hostToLocalTaskCount = localityToCount.toMap
697+
}
640698
}
641699

642700
/**

core/src/main/scala/org/apache/spark/Partitioner.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ object Partitioner {
5656
*/
5757
def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
5858
val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
59-
for (r <- bySize if r.partitioner.isDefined) {
59+
for (r <- bySize if r.partitioner.isDefined && r.partitioner.get.numPartitions > 0) {
6060
return r.partitioner.get
6161
}
6262
if (rdd.context.conf.contains("spark.default.parallelism")) {

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 19 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1382,16 +1382,29 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
13821382
}
13831383

13841384
/**
1385-
* Express a preference to the cluster manager for a given total number of executors.
1386-
* This can result in canceling pending requests or filing additional requests.
1387-
* This is currently only supported in YARN mode. Return whether the request is received.
1388-
*/
1389-
private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = {
1385+
* Update the cluster manager on our scheduling needs. Three bits of information are included
1386+
* to help it make decisions.
1387+
* @param numExecutors The total number of executors we'd like to have. The cluster manager
1388+
* shouldn't kill any running executor to reach this number, but,
1389+
* if all existing executors were to die, this is the number of executors
1390+
* we'd want to be allocated.
1391+
* @param localityAwareTasks The number of tasks in all active stages that have a locality
1392+
* preferences. This includes running, pending, and completed tasks.
1393+
* @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
1394+
* that would like to like to run on that host.
1395+
* This includes running, pending, and completed tasks.
1396+
* @return whether the request is acknowledged by the cluster manager.
1397+
*/
1398+
private[spark] override def requestTotalExecutors(
1399+
numExecutors: Int,
1400+
localityAwareTasks: Int,
1401+
hostToLocalTaskCount: scala.collection.immutable.Map[String, Int]
1402+
): Boolean = {
13901403
assert(supportDynamicAllocation,
13911404
"Requesting executors is currently only supported in YARN and Mesos modes")
13921405
schedulerBackend match {
13931406
case b: CoarseGrainedSchedulerBackend =>
1394-
b.requestTotalExecutors(numExecutors)
1407+
b.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount)
13951408
case _ =>
13961409
logWarning("Requesting executors is only supported in coarse-grained mode")
13971410
false

0 commit comments

Comments
 (0)