-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-4183] Close transport-related resources between SparkContexts #3053
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1178,6 +1178,10 @@ private[spark] class BlockManager( | |
|
||
def stop(): Unit = { | ||
blockTransferService.close() | ||
if (shuffleClient ne blockTransferService) { | ||
// Closing should be idempotent, but maybe not for the NioBlockTransferService. | ||
shuffleClient.close() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This and the other leaks would only leak during the ExternalShuffleIntegrationSuite, which only creates a handful of SparkContexts. |
||
} | ||
diskBlockManager.stop() | ||
actorSystem.stop(slaveActor) | ||
blockInfo.clear() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,7 +25,7 @@ import org.apache.spark.storage.BlockManagerId | |
/** | ||
* Test add and remove behavior of ExecutorAllocationManager. | ||
*/ | ||
class ExecutorAllocationManagerSuite extends FunSuite { | ||
class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @andrewor14 please take a look at the changes to this test There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM |
||
import ExecutorAllocationManager._ | ||
import ExecutorAllocationManagerSuite._ | ||
|
||
|
@@ -36,17 +36,21 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
.setAppName("test-executor-allocation-manager") | ||
.set("spark.dynamicAllocation.enabled", "true") | ||
intercept[SparkException] { new SparkContext(conf) } | ||
SparkEnv.get.stop() // cleanup the created environment | ||
|
||
// Only min | ||
val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") | ||
intercept[SparkException] { new SparkContext(conf1) } | ||
SparkEnv.get.stop() | ||
|
||
// Only max | ||
val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") | ||
intercept[SparkException] { new SparkContext(conf2) } | ||
SparkEnv.get.stop() | ||
|
||
// Both min and max, but min > max | ||
intercept[SparkException] { createSparkContext(2, 1) } | ||
SparkEnv.get.stop() | ||
|
||
// Both min and max, and min == max | ||
val sc1 = createSparkContext(1, 1) | ||
|
@@ -60,18 +64,17 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("starting state") { | ||
val sc = createSparkContext() | ||
sc = createSparkContext() | ||
val manager = sc.executorAllocationManager.get | ||
assert(numExecutorsPending(manager) === 0) | ||
assert(executorsPendingToRemove(manager).isEmpty) | ||
assert(executorIds(manager).isEmpty) | ||
assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) | ||
assert(removeTimes(manager).isEmpty) | ||
sc.stop() | ||
} | ||
|
||
test("add executors") { | ||
val sc = createSparkContext(1, 10) | ||
sc = createSparkContext(1, 10) | ||
val manager = sc.executorAllocationManager.get | ||
|
||
// Keep adding until the limit is reached | ||
|
@@ -112,11 +115,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
assert(addExecutors(manager) === 0) | ||
assert(numExecutorsPending(manager) === 6) | ||
assert(numExecutorsToAdd(manager) === 1) | ||
sc.stop() | ||
} | ||
|
||
test("remove executors") { | ||
val sc = createSparkContext(5, 10) | ||
sc = createSparkContext(5, 10) | ||
val manager = sc.executorAllocationManager.get | ||
(1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } | ||
|
||
|
@@ -163,11 +165,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
assert(executorsPendingToRemove(manager).isEmpty) | ||
assert(!removeExecutor(manager, "8")) | ||
assert(executorsPendingToRemove(manager).isEmpty) | ||
sc.stop() | ||
} | ||
|
||
test ("interleaving add and remove") { | ||
val sc = createSparkContext(5, 10) | ||
sc = createSparkContext(5, 10) | ||
val manager = sc.executorAllocationManager.get | ||
|
||
// Add a few executors | ||
|
@@ -232,11 +233,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
onExecutorAdded(manager, "15") | ||
onExecutorAdded(manager, "16") | ||
assert(executorIds(manager).size === 10) | ||
sc.stop() | ||
} | ||
|
||
test("starting/canceling add timer") { | ||
val sc = createSparkContext(2, 10) | ||
sc = createSparkContext(2, 10) | ||
val clock = new TestClock(8888L) | ||
val manager = sc.executorAllocationManager.get | ||
manager.setClock(clock) | ||
|
@@ -268,7 +268,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("starting/canceling remove timers") { | ||
val sc = createSparkContext(2, 10) | ||
sc = createSparkContext(2, 10) | ||
val clock = new TestClock(14444L) | ||
val manager = sc.executorAllocationManager.get | ||
manager.setClock(clock) | ||
|
@@ -313,7 +313,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("mock polling loop with no events") { | ||
val sc = createSparkContext(1, 20) | ||
sc = createSparkContext(1, 20) | ||
val manager = sc.executorAllocationManager.get | ||
val clock = new TestClock(2020L) | ||
manager.setClock(clock) | ||
|
@@ -339,7 +339,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("mock polling loop add behavior") { | ||
val sc = createSparkContext(1, 20) | ||
sc = createSparkContext(1, 20) | ||
val clock = new TestClock(2020L) | ||
val manager = sc.executorAllocationManager.get | ||
manager.setClock(clock) | ||
|
@@ -388,7 +388,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("mock polling loop remove behavior") { | ||
val sc = createSparkContext(1, 20) | ||
sc = createSparkContext(1, 20) | ||
val clock = new TestClock(2020L) | ||
val manager = sc.executorAllocationManager.get | ||
manager.setClock(clock) | ||
|
@@ -449,7 +449,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("listeners trigger add executors correctly") { | ||
val sc = createSparkContext(2, 10) | ||
sc = createSparkContext(2, 10) | ||
val manager = sc.executorAllocationManager.get | ||
assert(addTime(manager) === NOT_SET) | ||
|
||
|
@@ -479,7 +479,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("listeners trigger remove executors correctly") { | ||
val sc = createSparkContext(2, 10) | ||
sc = createSparkContext(2, 10) | ||
val manager = sc.executorAllocationManager.get | ||
assert(removeTimes(manager).isEmpty) | ||
|
||
|
@@ -510,7 +510,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { | |
} | ||
|
||
test("listeners trigger add and remove executor callbacks correctly") { | ||
val sc = createSparkContext(2, 10) | ||
sc = createSparkContext(2, 10) | ||
val manager = sc.executorAllocationManager.get | ||
assert(executorIds(manager).isEmpty) | ||
assert(removeTimes(manager).isEmpty) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,7 +28,7 @@ import org.apache.spark.shuffle.FetchFailedException | |
import org.apache.spark.storage.BlockManagerId | ||
import org.apache.spark.util.AkkaUtils | ||
|
||
class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | ||
class MapOutputTrackerSuite extends FunSuite { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @pwendell maybe take a look at the changes here? No one in particular is responsible for this test, I think. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since the shutdown code isn't in a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe this is less of a concern, though, since I guess we're more worried about leaked resources from a passing test causing a subsequent test to fail. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, either would be better, but since each test creates a different set of things, and sometimes calling stop() throws an exception, I decided to just do it in the non-failing case for this PR. |
||
private val conf = new SparkConf | ||
|
||
test("master start and stop") { | ||
|
@@ -37,6 +37,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
tracker.trackerActor = | ||
actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) | ||
tracker.stop() | ||
actorSystem.shutdown() | ||
} | ||
|
||
test("master register shuffle and fetch") { | ||
|
@@ -56,6 +57,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), | ||
(BlockManagerId("b", "hostB", 1000), size10000))) | ||
tracker.stop() | ||
actorSystem.shutdown() | ||
} | ||
|
||
test("master register and unregister shuffle") { | ||
|
@@ -74,6 +76,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
tracker.unregisterShuffle(10) | ||
assert(!tracker.containsShuffle(10)) | ||
assert(tracker.getServerStatuses(10, 0).isEmpty) | ||
|
||
tracker.stop() | ||
actorSystem.shutdown() | ||
} | ||
|
||
test("master register shuffle and unregister map output and fetch") { | ||
|
@@ -97,6 +102,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
// this should cause it to fail, and the scheduler will ignore the failure due to the | ||
// stage already being aborted. | ||
intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) } | ||
|
||
tracker.stop() | ||
actorSystem.shutdown() | ||
} | ||
|
||
test("remote fetch") { | ||
|
@@ -136,6 +144,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
|
||
// failure should be cached | ||
intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } | ||
|
||
masterTracker.stop() | ||
slaveTracker.stop() | ||
actorSystem.shutdown() | ||
slaveSystem.shutdown() | ||
} | ||
|
||
test("remote fetch below akka frame size") { | ||
|
@@ -154,6 +167,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
masterTracker.registerMapOutput(10, 0, MapStatus( | ||
BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) | ||
masterActor.receive(GetMapOutputStatuses(10)) | ||
|
||
// masterTracker.stop() // this throws an exception | ||
actorSystem.shutdown() | ||
} | ||
|
||
test("remote fetch exceeds akka frame size") { | ||
|
@@ -176,5 +192,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { | |
BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) | ||
} | ||
intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } | ||
|
||
// masterTracker.stop() // this throws an exception | ||
actorSystem.shutdown() | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -49,6 +49,7 @@ public class TransportServer implements Closeable { | |
private ChannelFuture channelFuture; | ||
private int port = -1; | ||
|
||
/** Creates a TransportServer that binds to the given port, or to any available if 0. */ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This was a comment @rxin asked for, probably won't hurt anything. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 :) |
||
public TransportServer(TransportContext context, int portToBind) { | ||
this.context = context; | ||
this.conf = context.getConf(); | ||
|
@@ -67,7 +68,7 @@ private void init(int portToBind) { | |
|
||
IOMode ioMode = IOMode.valueOf(conf.ioMode()); | ||
EventLoopGroup bossGroup = | ||
NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); | ||
NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); | ||
EventLoopGroup workerGroup = bossGroup; | ||
|
||
bootstrap = new ServerBootstrap() | ||
|
@@ -105,7 +106,7 @@ protected void initChannel(SocketChannel ch) throws Exception { | |
@Override | ||
public void close() { | ||
if (channelFuture != null) { | ||
// close is a local operation and should finish with milliseconds; timeout just to be safe | ||
// close is a local operation and should finish within milliseconds; timeout just to be safe | ||
channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS); | ||
channelFuture = null; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -46,6 +46,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w | |
after { | ||
if (ssc != null) { | ||
ssc.stop() | ||
if (ssc.sc != null) { | ||
// Calling ssc.stop() does not always stop the associated SparkContext. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @tdas I have reverted the changes, though you can see from the fact that we have to do this that the API is not clean. |
||
ssc.sc.stop() | ||
} | ||
ssc = null | ||
} | ||
if (sc != null) { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is the most significant leak, we'd leak one event loop per SparkContext