Skip to content

Commit aa87c40

Browse files
committed
Fixed merge conflicts
2 parents 9d66e81 + 67d468f commit aa87c40

File tree

236 files changed

+4147
-3097
lines changed

Some content is hidden

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

236 files changed

+4147
-3097
lines changed

LICENSE

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -265,7 +265,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
265265
(New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf)
266266
(The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net)
267267
(The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net)
268-
(The New BSD License) Py4J (net.sf.py4j:py4j:0.8.2.1 - http://py4j.sourceforge.net/)
268+
(The New BSD License) Py4J (net.sf.py4j:py4j:0.9 - http://py4j.sourceforge.net/)
269269
(Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/)
270270
(BSD licence) sbt and sbt-launch-lib.bash
271271
(BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE)

R/pkg/R/mllib.R

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,11 +45,12 @@ setClass("PipelineModel", representation(model = "jobj"))
4545
#' summary(model)
4646
#'}
4747
setMethod("glm", signature(formula = "formula", family = "ANY", data = "DataFrame"),
48-
function(formula, family = c("gaussian", "binomial"), data, lambda = 0, alpha = 0) {
48+
function(formula, family = c("gaussian", "binomial"), data, lambda = 0, alpha = 0,
49+
solver = "auto") {
4950
family <- match.arg(family)
5051
model <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers",
5152
"fitRModelFormula", deparse(formula), data@sdf, family, lambda,
52-
alpha)
53+
alpha, solver)
5354
return(new("PipelineModel", model = model))
5455
})
5556

R/pkg/inst/tests/test_mllib.R

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ test_that("feature interaction vs native glm", {
5959

6060
test_that("summary coefficients match with native glm", {
6161
training <- createDataFrame(sqlContext, iris)
62-
stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training))
62+
stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training, solver = "l-bfgs"))
6363
coefs <- as.vector(stats$coefficients)
6464
rCoefs <- as.vector(coef(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)))
6565
expect_true(all(abs(rCoefs - coefs) < 1e-6))

bin/pyspark

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ export PYSPARK_PYTHON
6565

6666
# Add the PySpark classes to the Python path:
6767
export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH"
68-
export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH"
68+
export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.9-src.zip:$PYTHONPATH"
6969

7070
# Load the PySpark shell.py script when ./pyspark is used interactively:
7171
export OLD_PYTHONSTARTUP="$PYTHONSTARTUP"

bin/pyspark2.cmd

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" (
3030
)
3131

3232
set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH%
33-
set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH%
33+
set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9-src.zip;%PYTHONPATH%
3434

3535
set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
3636
set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py

core/pom.xml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -339,7 +339,7 @@
339339
<dependency>
340340
<groupId>net.razorvine</groupId>
341341
<artifactId>pyrolite</artifactId>
342-
<version>4.4</version>
342+
<version>4.9</version>
343343
<exclusions>
344344
<exclusion>
345345
<groupId>net.razorvine</groupId>
@@ -350,7 +350,7 @@
350350
<dependency>
351351
<groupId>net.sf.py4j</groupId>
352352
<artifactId>py4j</artifactId>
353-
<version>0.8.2.1</version>
353+
<version>0.9</version>
354354
</dependency>
355355
<dependency>
356356
<groupId>org.apache.spark</groupId>

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,19 +21,18 @@ import org.apache.log4j.{LogManager, PropertyConfigurator}
2121
import org.slf4j.{Logger, LoggerFactory}
2222
import org.slf4j.impl.StaticLoggerBinder
2323

24-
import org.apache.spark.annotation.DeveloperApi
24+
import org.apache.spark.annotation.Private
2525
import org.apache.spark.util.Utils
2626

2727
/**
28-
* :: DeveloperApi ::
2928
* Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
3029
* logging messages at different levels using methods that only evaluate parameters lazily if the
3130
* log level is enabled.
3231
*
3332
* NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility.
3433
* This will likely be changed or removed in future releases.
3534
*/
36-
@DeveloperApi
35+
@Private
3736
trait Logging {
3837
// Make the log field transient so that objects with Logging can
3938
// be serialized and used on another machine

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -595,7 +595,9 @@ private[spark] object SparkConf extends Logging {
595595
"spark.rpc.lookupTimeout" -> Seq(
596596
AlternateConfig("spark.akka.lookupTimeout", "1.4")),
597597
"spark.streaming.fileStream.minRememberDuration" -> Seq(
598-
AlternateConfig("spark.streaming.minRememberDuration", "1.5"))
598+
AlternateConfig("spark.streaming.minRememberDuration", "1.5")),
599+
"spark.yarn.max.executor.failures" -> Seq(
600+
AlternateConfig("spark.yarn.max.worker.failures", "1.5"))
599601
)
600602

601603
/**

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

Lines changed: 6 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -90,11 +90,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
9090
// NOTE: this must be placed at the beginning of the SparkContext constructor.
9191
SparkContext.markPartiallyConstructed(this, allowMultipleContexts)
9292

93-
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
94-
// etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
95-
// contains a map from hostname to a list of input format splits on the host.
96-
private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()
97-
9893
val startTime = System.currentTimeMillis()
9994

10095
private[spark] val stopped: AtomicBoolean = new AtomicBoolean(false)
@@ -116,16 +111,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
116111
* Alternative constructor for setting preferred locations where Spark will create executors.
117112
*
118113
* @param config a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
119-
* @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on.
120-
* Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
121-
* from a list of input files or InputFormats for the application.
114+
* @param preferredNodeLocationData not used. Left for backward compatibility.
122115
*/
123116
@deprecated("Passing in preferred locations has no effect at all, see SPARK-8949", "1.5.0")
124117
@DeveloperApi
125118
def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = {
126119
this(config)
127120
logWarning("Passing in preferred locations has no effect at all, see SPARK-8949")
128-
this.preferredNodeLocationData = preferredNodeLocationData
129121
}
130122

131123
/**
@@ -147,10 +139,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
147139
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file
148140
* system or HDFS, HTTP, HTTPS, or FTP URLs.
149141
* @param environment Environment variables to set on worker nodes.
150-
* @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on.
151-
* Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
152-
* from a list of input files or InputFormats for the application.
142+
* @param preferredNodeLocationData not used. Left for backward compatibility.
153143
*/
144+
@deprecated("Passing in preferred locations has no effect at all, see SPARK-10921", "1.6.0")
154145
def this(
155146
master: String,
156147
appName: String,
@@ -163,7 +154,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
163154
if (preferredNodeLocationData.nonEmpty) {
164155
logWarning("Passing in preferred locations has no effect at all, see SPARK-8949")
165156
}
166-
this.preferredNodeLocationData = preferredNodeLocationData
167157
}
168158

169159
// NOTE: The below constructors could be consolidated using default arguments. Due to
@@ -177,7 +167,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
177167
* @param appName A name for your application, to display on the cluster web UI.
178168
*/
179169
private[spark] def this(master: String, appName: String) =
180-
this(master, appName, null, Nil, Map(), Map())
170+
this(master, appName, null, Nil, Map())
181171

182172
/**
183173
* Alternative constructor that allows setting common Spark properties directly
@@ -187,7 +177,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
187177
* @param sparkHome Location where Spark is installed on cluster nodes.
188178
*/
189179
private[spark] def this(master: String, appName: String, sparkHome: String) =
190-
this(master, appName, sparkHome, Nil, Map(), Map())
180+
this(master, appName, sparkHome, Nil, Map())
191181

192182
/**
193183
* Alternative constructor that allows setting common Spark properties directly
@@ -199,7 +189,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
199189
* system or HDFS, HTTP, HTTPS, or FTP URLs.
200190
*/
201191
private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) =
202-
this(master, appName, sparkHome, jars, Map(), Map())
192+
this(master, appName, sparkHome, jars, Map())
203193

204194
// log out Spark Version in Spark driver log
205195
logInfo(s"Running Spark version $SPARK_VERSION")

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

Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,10 +24,14 @@ import java.util.Arrays
2424
import java.util.jar.{JarEntry, JarOutputStream}
2525

2626
import scala.collection.JavaConverters._
27+
import scala.collection.mutable
28+
import scala.collection.mutable.ArrayBuffer
2729

2830
import com.google.common.io.{ByteStreams, Files}
2931
import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}
3032

33+
import org.apache.spark.executor.TaskMetrics
34+
import org.apache.spark.scheduler._
3135
import org.apache.spark.util.Utils
3236

3337
/**
@@ -154,4 +158,51 @@ private[spark] object TestUtils {
154158
" @Override public String toString() { return \"" + toStringValue + "\"; }}")
155159
createCompiledClass(className, destDir, sourceFile, classpathUrls)
156160
}
161+
162+
/**
163+
* Run some code involving jobs submitted to the given context and assert that the jobs spilled.
164+
*/
165+
def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
166+
val spillListener = new SpillListener
167+
sc.addSparkListener(spillListener)
168+
body
169+
assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
170+
}
171+
172+
/**
173+
* Run some code involving jobs submitted to the given context and assert that the jobs
174+
* did not spill.
175+
*/
176+
def assertNotSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
177+
val spillListener = new SpillListener
178+
sc.addSparkListener(spillListener)
179+
body
180+
assert(spillListener.numSpilledStages == 0, s"expected $identifier to not spill, but did")
181+
}
182+
183+
}
184+
185+
186+
/**
187+
* A [[SparkListener]] that detects whether spills have occurred in Spark jobs.
188+
*/
189+
private class SpillListener extends SparkListener {
190+
private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]]
191+
private val spilledStageIds = new mutable.HashSet[Int]
192+
193+
def numSpilledStages: Int = spilledStageIds.size
194+
195+
override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
196+
stageIdToTaskMetrics.getOrElseUpdate(
197+
taskEnd.stageId, new ArrayBuffer[TaskMetrics]) += taskEnd.taskMetrics
198+
}
199+
200+
override def onStageCompleted(stageComplete: SparkListenerStageCompleted): Unit = {
201+
val stageId = stageComplete.stageInfo.stageId
202+
val metrics = stageIdToTaskMetrics.remove(stageId).toSeq.flatten
203+
val spilled = metrics.map(_.memoryBytesSpilled).sum > 0
204+
if (spilled) {
205+
spilledStageIds += stageId
206+
}
207+
}
157208
}

0 commit comments

Comments
 (0)