From 4d5c12aa1c54c49377a4bafe3bcc4993d5e1a552 Mon Sep 17 00:00:00 2001 From: Lars Albertsson Date: Wed, 11 Jun 2014 10:54:42 -0700 Subject: SPARK-2113: awaitTermination() after stop() will hang in Spark Stremaing Author: Lars Albertsson Closes #1001 from lallea/contextwaiter_stopped and squashes the following commits: 93cd314 [Lars Albertsson] Mend StreamingContext stop() followed by awaitTermination(). --- streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala | 1 + 1 file changed, 1 insertion(+) (limited to 'streaming/src/main') diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala index 86753360a0..a0aeacbc73 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala @@ -27,6 +27,7 @@ private[streaming] class ContextWaiter { } def notifyStop() = synchronized { + stopped = true notifyAll() } -- cgit v1.2.3