Skip to content

Commit 0e29eac

Browse files
committed
Merge branch 'master' of https://github.com/apache/spark into SPARK-1712_new
2 parents 3ea1ca1 + 5c0dafc commit 0e29eac

File tree

92 files changed

+1048
-453
lines changed

Some content is hidden

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

92 files changed

+1048
-453
lines changed

bagel/src/test/resources/log4j.properties

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@
1919
log4j.rootCategory=INFO, file
2020
log4j.appender.file=org.apache.log4j.FileAppender
2121
log4j.appender.file.append=false
22-
log4j.appender.file.file=bagel/target/unit-tests.log
22+
log4j.appender.file.file=target/unit-tests.log
2323
log4j.appender.file.layout=org.apache.log4j.PatternLayout
2424
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
2525

bin/spark-shell.cmd

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and
1717
rem limitations under the License.
1818
rem
1919

20-
rem Find the path of sbin
21-
set BIN=%~dp0..\bin\
20+
set SPARK_HOME=%~dp0..
2221

23-
cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %*
22+
cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main

bin/spark-submit.cmd

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
1+
@echo off
2+
3+
rem
4+
rem Licensed to the Apache Software Foundation (ASF) under one or more
5+
rem contributor license agreements. See the NOTICE file distributed with
6+
rem this work for additional information regarding copyright ownership.
7+
rem The ASF licenses this file to You under the Apache License, Version 2.0
8+
rem (the "License"); you may not use this file except in compliance with
9+
rem the License. You may obtain a copy of the License at
10+
rem
11+
rem http://www.apache.org/licenses/LICENSE-2.0
12+
rem
13+
rem Unless required by applicable law or agreed to in writing, software
14+
rem distributed under the License is distributed on an "AS IS" BASIS,
15+
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16+
rem See the License for the specific language governing permissions and
17+
rem limitations under the License.
18+
rem
19+
20+
set SPARK_HOME=%~dp0..
21+
set ORIG_ARGS=%*
22+
23+
rem Clear the values of all variables used
24+
set DEPLOY_MODE=
25+
set DRIVER_MEMORY=
26+
set SPARK_SUBMIT_LIBRARY_PATH=
27+
set SPARK_SUBMIT_CLASSPATH=
28+
set SPARK_SUBMIT_OPTS=
29+
set SPARK_DRIVER_MEMORY=
30+
31+
:loop
32+
if [%1] == [] goto continue
33+
if [%1] == [--deploy-mode] (
34+
set DEPLOY_MODE=%2
35+
) else if [%1] == [--driver-memory] (
36+
set DRIVER_MEMORY=%2
37+
) else if [%1] == [--driver-library-path] (
38+
set SPARK_SUBMIT_LIBRARY_PATH=%2
39+
) else if [%1] == [--driver-class-path] (
40+
set SPARK_SUBMIT_CLASSPATH=%2
41+
) else if [%1] == [--driver-java-options] (
42+
set SPARK_SUBMIT_OPTS=%2
43+
)
44+
shift
45+
goto loop
46+
:continue
47+
48+
if [%DEPLOY_MODE%] == [] (
49+
set DEPLOY_MODE=client
50+
)
51+
52+
if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] (
53+
set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY%
54+
)
55+
56+
cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS%

conf/spark-defaults.conf.template

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
# This is useful for setting default environmental settings.
33

44
# Example:
5-
# spark.master spark://master:7077
5+
# spark.master spark://master:7077
66
# spark.eventLog.enabled true
77
# spark.eventLog.dir hdfs://namenode:8021/directory
8+
# spark.serializer org.apache.spark.serializer.KryoSerializer

conf/spark-env.sh.template

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,14 +30,14 @@
3030

3131
# Options for the daemons used in the standalone deploy mode:
3232
# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname
33-
# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports
33+
# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master
3434
# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y")
3535
# - SPARK_WORKER_CORES, to set the number of cores to use on this machine
3636
# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g)
37-
# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
37+
# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker
3838
# - SPARK_WORKER_INSTANCES, to set the number of worker processes per node
3939
# - SPARK_WORKER_DIR, to set the working directory of worker processes
4040
# - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y")
4141
# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y")
42-
# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y")
42+
# - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y")
4343
# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers

core/pom.xml

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -140,6 +140,7 @@
140140
<dependency>
141141
<groupId>org.apache.mesos</groupId>
142142
<artifactId>mesos</artifactId>
143+
<classifier>${mesos.classifier}</classifier>
143144
</dependency>
144145
<dependency>
145146
<groupId>io.netty</groupId>
@@ -322,7 +323,7 @@
322323
</configuration>
323324
</plugin>
324325
</plugins>
325-
326+
326327
<resources>
327328
<resource>
328329
<directory>src/main/resources</directory>

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

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
2323

2424
import org.apache.spark.broadcast.Broadcast
2525
import org.apache.spark.rdd.RDD
26+
import org.apache.spark.util.Utils
2627

2728
/**
2829
* Classes that represent cleaning tasks.
@@ -110,7 +111,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
110111
}
111112

112113
/** Keep cleaning RDD, shuffle, and broadcast state. */
113-
private def keepCleaning() {
114+
private def keepCleaning(): Unit = Utils.logUncaughtExceptions {
114115
while (!stopped) {
115116
try {
116117
val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT))
@@ -128,7 +129,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
128129
}
129130
}
130131
} catch {
131-
case t: Throwable => logError("Error in cleaning thread", t)
132+
case e: Exception => logError("Error in cleaning thread", e)
132133
}
133134
}
134135
}
@@ -141,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
141142
listeners.foreach(_.rddCleaned(rddId))
142143
logInfo("Cleaned RDD " + rddId)
143144
} catch {
144-
case t: Throwable => logError("Error cleaning RDD " + rddId, t)
145+
case e: Exception => logError("Error cleaning RDD " + rddId, e)
145146
}
146147
}
147148

@@ -154,7 +155,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
154155
listeners.foreach(_.shuffleCleaned(shuffleId))
155156
logInfo("Cleaned shuffle " + shuffleId)
156157
} catch {
157-
case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t)
158+
case e: Exception => logError("Error cleaning shuffle " + shuffleId, e)
158159
}
159160
}
160161

@@ -166,7 +167,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
166167
listeners.foreach(_.broadcastCleaned(broadcastId))
167168
logInfo("Cleaned broadcast " + broadcastId)
168169
} catch {
169-
case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t)
170+
case e: Exception => logError("Error cleaning broadcast " + broadcastId, e)
170171
}
171172
}
172173

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -247,7 +247,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
247247
| - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
248248
| - ./spark-submit with --driver-java-options to set -X options for a driver
249249
| - spark.executor.extraJavaOptions to set -X options for executors
250-
| - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker)
250+
| - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker)
251251
""".stripMargin
252252
logError(error)
253253

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

Lines changed: 11 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -50,15 +50,13 @@ import org.apache.spark.ui.SparkUI
5050
import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils}
5151

5252
/**
53-
* :: DeveloperApi ::
5453
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
5554
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
5655
*
5756
* @param config a Spark Config object describing the application configuration. Any settings in
5857
* this config overrides the default configs as well as system properties.
5958
*/
6059

61-
@DeveloperApi
6260
class SparkContext(config: SparkConf) extends Logging {
6361

6462
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
@@ -276,27 +274,26 @@ class SparkContext(config: SparkConf) extends Logging {
276274
.getOrElse(512)
277275

278276
// Environment variables to pass to our executors.
279-
// NOTE: This should only be used for test related settings.
280-
private[spark] val testExecutorEnvs = HashMap[String, String]()
277+
private[spark] val executorEnvs = HashMap[String, String]()
281278

282279
// Convert java options to env vars as a work around
283280
// since we can't set env vars directly in sbt.
284281
for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing"))
285282
value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
286-
testExecutorEnvs(envKey) = value
283+
executorEnvs(envKey) = value
287284
}
288285
// The Mesos scheduler backend relies on this environment variable to set executor memory.
289286
// TODO: Set this only in the Mesos scheduler.
290-
testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m"
291-
testExecutorEnvs ++= conf.getExecutorEnv
287+
executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m"
288+
executorEnvs ++= conf.getExecutorEnv
292289

293290
// Set SPARK_USER for user who is running SparkContext.
294291
val sparkUser = Option {
295292
Option(System.getProperty("user.name")).getOrElse(System.getenv("SPARK_USER"))
296293
}.getOrElse {
297294
SparkContext.SPARK_UNKNOWN_USER
298295
}
299-
testExecutorEnvs("SPARK_USER") = sparkUser
296+
executorEnvs("SPARK_USER") = sparkUser
300297

301298
// Create and start the scheduler
302299
private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master)
@@ -1494,8 +1491,8 @@ object SparkContext extends Logging {
14941491
} catch {
14951492
// TODO: Enumerate the exact reasons why it can fail
14961493
// But irrespective of it, it means we cannot proceed !
1497-
case th: Throwable => {
1498-
throw new SparkException("YARN mode not available ?", th)
1494+
case e: Exception => {
1495+
throw new SparkException("YARN mode not available ?", e)
14991496
}
15001497
}
15011498
val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
@@ -1510,8 +1507,8 @@ object SparkContext extends Logging {
15101507
cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
15111508

15121509
} catch {
1513-
case th: Throwable => {
1514-
throw new SparkException("YARN mode not available ?", th)
1510+
case e: Exception => {
1511+
throw new SparkException("YARN mode not available ?", e)
15151512
}
15161513
}
15171514

@@ -1521,8 +1518,8 @@ object SparkContext extends Logging {
15211518
val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
15221519
cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
15231520
} catch {
1524-
case th: Throwable => {
1525-
throw new SparkException("YARN mode not available ?", th)
1521+
case e: Exception => {
1522+
throw new SparkException("YARN mode not available ?", e)
15261523
}
15271524
}
15281525

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ private[spark] object TestUtils {
4343
*/
4444
def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = {
4545
val tempDir = Files.createTempDir()
46+
tempDir.deleteOnExit()
4647
val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value)
4748
val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis()))
4849
createJar(files, jarFile)

0 commit comments

Comments
 (0)