Skip to content

Commit f1cb9b3

Browse files
author
Marcelo Vanzin
committed
Merge branch 'master' into SPARK-4705
2 parents c14ec19 + cc48e63 commit f1cb9b3

File tree

191 files changed

+3055
-1246
lines changed

Some content is hidden

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

191 files changed

+3055
-1246
lines changed

CONTRIBUTING.md

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,16 @@
11
## Contributing to Spark
22

3-
Contributions via GitHub pull requests are gladly accepted from their original
4-
author. Along with any pull requests, please state that the contribution is
5-
your original work and that you license the work to the project under the
6-
project's open source license. Whether or not you state this explicitly, by
7-
submitting any copyrighted material via pull request, email, or other means
8-
you agree to license the material under the project's open source license and
9-
warrant that you have the legal authority to do so.
3+
*Before opening a pull request*, review the
4+
[Contributing to Spark wiki](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark).
5+
It lists steps that are required before creating a PR. In particular, consider:
6+
7+
- Is the change important and ready enough to ask the community to spend time reviewing?
8+
- Have you searched for existing, related JIRAs and pull requests?
9+
- Is this a new feature that can stand alone as a package on http://spark-packages.org ?
10+
- Is the change being proposed clearly explained and motivated?
1011

11-
Please see the [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark)
12-
for more information.
12+
When you contribute code, you affirm that the contribution is your original work and that you
13+
license the work to the project under the project's open source license. Whether or not you
14+
state this explicitly, by submitting any copyrighted material via pull request, email, or
15+
other means you agree to license the material under the project's open source license and
16+
warrant that you have the legal authority to do so.

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,10 +44,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
4444
blockManager.get(key) match {
4545
case Some(blockResult) =>
4646
// Partition is already materialized, so just return its values
47-
val inputMetrics = blockResult.inputMetrics
4847
val existingMetrics = context.taskMetrics
49-
.getInputMetricsForReadMethod(inputMetrics.readMethod)
50-
existingMetrics.incBytesRead(inputMetrics.bytesRead)
48+
.getInputMetricsForReadMethod(blockResult.readMethod)
49+
existingMetrics.incBytesRead(blockResult.bytes)
5150

5251
val iter = blockResult.data.asInstanceOf[Iterator[T]]
5352
new InterruptibleIterator[T](context, iter) {

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

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,12 @@
1717

1818
package org.apache.spark
1919

20-
import java.util.concurrent.{Executors, TimeUnit}
20+
import java.util.concurrent.TimeUnit
2121

2222
import scala.collection.mutable
2323

2424
import org.apache.spark.scheduler._
25-
import org.apache.spark.util.{Clock, SystemClock, Utils}
25+
import org.apache.spark.util.{ThreadUtils, Clock, SystemClock, Utils}
2626

2727
/**
2828
* An agent that dynamically allocates and removes executors based on the workload.
@@ -132,8 +132,8 @@ private[spark] class ExecutorAllocationManager(
132132
private val listener = new ExecutorAllocationListener
133133

134134
// Executor that handles the scheduling task.
135-
private val executor = Executors.newSingleThreadScheduledExecutor(
136-
Utils.namedThreadFactory("spark-dynamic-executor-allocation"))
135+
private val executor =
136+
ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation")
137137

138138
/**
139139
* Verify that the settings specified through the config are valid.

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

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,15 @@
1717

1818
package org.apache.spark
1919

20-
import java.util.concurrent.{ScheduledFuture, TimeUnit, Executors}
20+
import java.util.concurrent.{ScheduledFuture, TimeUnit}
2121

2222
import scala.collection.mutable
2323

2424
import org.apache.spark.executor.TaskMetrics
2525
import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext}
2626
import org.apache.spark.storage.BlockManagerId
2727
import org.apache.spark.scheduler.{SlaveLost, TaskScheduler}
28-
import org.apache.spark.util.Utils
28+
import org.apache.spark.util.{ThreadUtils, Utils}
2929

3030
/**
3131
* A heartbeat from executors to the driver. This is a shared message used by several internal
@@ -76,11 +76,10 @@ private[spark] class HeartbeatReceiver(sc: SparkContext)
7676

7777
private var timeoutCheckingTask: ScheduledFuture[_] = null
7878

79-
private val timeoutCheckingThread = Executors.newSingleThreadScheduledExecutor(
80-
Utils.namedThreadFactory("heartbeat-timeout-checking-thread"))
79+
private val timeoutCheckingThread =
80+
ThreadUtils.newDaemonSingleThreadScheduledExecutor("heartbeat-timeout-checking-thread")
8181

82-
private val killExecutorThread = Executors.newSingleThreadExecutor(
83-
Utils.namedThreadFactory("kill-executor-thread"))
82+
private val killExecutorThread = ThreadUtils.newDaemonSingleThreadExecutor("kill-executor-thread")
8483

8584
override def onStart(): Unit = {
8685
timeoutCheckingTask = timeoutCheckingThread.scheduleAtFixedRate(new Runnable {

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

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -431,7 +431,15 @@ private[spark] object SparkConf extends Logging {
431431
"spark.yarn.am.waitTime" -> Seq(
432432
AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3",
433433
// Translate old value to a duration, with 10s wait time per try.
434-
translation = s => s"${s.toLong * 10}s"))
434+
translation = s => s"${s.toLong * 10}s")),
435+
"spark.rpc.numRetries" -> Seq(
436+
AlternateConfig("spark.akka.num.retries", "1.4")),
437+
"spark.rpc.retry.wait" -> Seq(
438+
AlternateConfig("spark.akka.retry.wait", "1.4")),
439+
"spark.rpc.askTimeout" -> Seq(
440+
AlternateConfig("spark.akka.askTimeout", "1.4")),
441+
"spark.rpc.lookupTimeout" -> Seq(
442+
AlternateConfig("spark.akka.lookupTimeout", "1.4"))
435443
)
436444

437445
/**

core/src/main/scala/org/apache/spark/deploy/Client.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.log4j.{Level, Logger}
2727
import org.apache.spark.{Logging, SecurityManager, SparkConf}
2828
import org.apache.spark.deploy.DeployMessages._
2929
import org.apache.spark.deploy.master.{DriverState, Master}
30-
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils}
30+
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, Utils}
3131

3232
/**
3333
* Proxy that relays messages to the driver.
@@ -36,7 +36,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
3636
extends Actor with ActorLogReceive with Logging {
3737

3838
var masterActor: ActorSelection = _
39-
val timeout = AkkaUtils.askTimeout(conf)
39+
val timeout = RpcUtils.askTimeout(conf)
4040

4141
override def preStart(): Unit = {
4242
masterActor = context.actorSelection(
@@ -155,7 +155,7 @@ object Client {
155155
if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
156156
conf.set("spark.akka.logLifecycleEvents", "true")
157157
}
158-
conf.set("spark.akka.askTimeout", "10")
158+
conf.set("spark.rpc.askTimeout", "10")
159159
conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING"))
160160
Logger.getRootLogger.setLevel(driverArgs.logLevel)
161161

core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf}
3030
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
3131
import org.apache.spark.deploy.DeployMessages._
3232
import org.apache.spark.deploy.master.Master
33-
import org.apache.spark.util.{ActorLogReceive, Utils, AkkaUtils}
33+
import org.apache.spark.util.{ActorLogReceive, RpcUtils, Utils, AkkaUtils}
3434

3535
/**
3636
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
@@ -193,7 +193,7 @@ private[spark] class AppClient(
193193
def stop() {
194194
if (actor != null) {
195195
try {
196-
val timeout = AkkaUtils.askTimeout(conf)
196+
val timeout = RpcUtils.askTimeout(conf)
197197
val future = actor.ask(StopAppClient)(timeout)
198198
Await.result(future, timeout)
199199
} catch {

core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.deploy.SparkHadoopUtil
3232
import org.apache.spark.io.CompressionCodec
3333
import org.apache.spark.scheduler._
3434
import org.apache.spark.ui.SparkUI
35-
import org.apache.spark.util.Utils
35+
import org.apache.spark.util.{ThreadUtils, Utils}
3636
import org.apache.spark.{Logging, SecurityManager, SparkConf}
3737

3838

@@ -99,7 +99,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
9999
*/
100100
private val replayExecutor: ExecutorService = {
101101
if (!conf.contains("spark.testing")) {
102-
Executors.newSingleThreadExecutor(Utils.namedThreadFactory("log-replay-executor"))
102+
ThreadUtils.newDaemonSingleThreadExecutor("log-replay-executor")
103103
} else {
104104
MoreExecutors.sameThreadExecutor()
105105
}

core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf}
2727
import org.apache.spark.deploy.SparkHadoopUtil
2828
import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
2929
import org.apache.spark.ui.JettyUtils._
30-
import org.apache.spark.util.SignalLogger
30+
import org.apache.spark.util.{SignalLogger, Utils}
3131

3232
/**
3333
* A web server that renders SparkUIs of completed applications.
@@ -203,9 +203,7 @@ object HistoryServer extends Logging {
203203
val server = new HistoryServer(conf, provider, securityManager, port)
204204
server.bind()
205205

206-
Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") {
207-
override def run(): Unit = server.stop()
208-
})
206+
Utils.addShutdownHook { () => server.stop() }
209207

210208
// Wait until the end of the world... or if the HistoryServer process is manually stopped
211209
while(true) { Thread.sleep(Int.MaxValue) }

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer
4747
import org.apache.spark.metrics.MetricsSystem
4848
import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus}
4949
import org.apache.spark.ui.SparkUI
50-
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils}
50+
import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, SignalLogger, Utils}
5151

5252
private[master] class Master(
5353
host: String,
@@ -931,7 +931,7 @@ private[deploy] object Master extends Logging {
931931
securityManager = securityMgr)
932932
val actor = actorSystem.actorOf(
933933
Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName)
934-
val timeout = AkkaUtils.askTimeout(conf)
934+
val timeout = RpcUtils.askTimeout(conf)
935935
val portsRequest = actor.ask(BoundPortsRequest)(timeout)
936936
val portsResponse = Await.result(portsRequest, timeout).asInstanceOf[BoundPortsResponse]
937937
(actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort)

0 commit comments

Comments
 (0)