Skip to content

Commit

Permalink
[SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnv
Browse files Browse the repository at this point in the history
`rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever.

This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv.

Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it.

Author: Shixiong Zhu <[email protected]>

Closes #11031 from zsxwing/awaitTermination.
  • Loading branch information
zsxwing authored and rxin committed Feb 3, 2016
1 parent 0557146 commit 335f10e
Show file tree
Hide file tree
Showing 2 changed files with 3 additions and 2 deletions.
1 change: 1 addition & 0 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@ class SparkEnv (
metricsSystem.stop()
outputCommitCoordinator.stop()
rpcEnv.shutdown()
rpcEnv.awaitTermination()

// Note that blockTransferService is stopped by BlockManager since it is started by it.

Expand Down
4 changes: 2 additions & 2 deletions python/pyspark/streaming/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -1013,12 +1013,12 @@ def setUp(self):
self._kafkaTestUtils.setup()

def tearDown(self):
super(KafkaStreamTests, self).tearDown()

if self._kafkaTestUtils is not None:
self._kafkaTestUtils.teardown()
self._kafkaTestUtils = None

super(KafkaStreamTests, self).tearDown()

def _randomTopic(self):
return "topic-%d" % random.randint(0, 10000)

Expand Down

0 comments on commit 335f10e

Please sign in to comment.