Skip to content
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-4813][Streaming] Fix the issue that ContextWaiter didn't handle 'spurious wakeup' #3661

Closed
wants to merge 3 commits into from

Conversation

zsxwing
Copy link
Member

@zsxwing zsxwing commented Dec 10, 2014

Used Condition to rewrite ContextWaiter because it provides a convenient API awaitNanos for timeout.

@SparkQA
Copy link

SparkQA commented Dec 10, 2014

Test build #24302 has started for PR 3661 at commit e06bd4f.

  • This patch merges cleanly.

private val lock = new ReentrantLock()
private val condition = lock.newCondition()

@GuardedBy("lock")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor point - these are not in the JDK but in a Findbugs library for JSR-305. It's not used in Spark, and happens to be a dependency now. Maybe not worth using it just 1 place?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe not worth using it just 1 place?

So which one do you prefer?

  1. Use comments to describe such information.
  2. Use GuardedBy from now on.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In addition, now Findbugs does not recognize GuardedBy in Scala codes.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, I turned to GuardedBy because @aarondav asked me to do it in #3634

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that's why I brought it up. It's not actually a standard Java annotation (unless someone tells me it just turned up in 8 or something) but part of JSR-305. This is a dependency of Spark core at the moment, but none of the annotations are used. I think we should just not use them instead of using this lib in 1 place.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK. I changed them to comments.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In general, I'm a fan of the FindBugs annotations. I've had trouble getting the various analysis tools to work well with them in Scala, though. +1 to this commenting convention, though; this is very helpful.

@SparkQA
Copy link

SparkQA commented Dec 10, 2014

Test build #24305 has started for PR 3661 at commit be42bcf.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 10, 2014

Test build #24302 has finished for PR 3661 at commit e06bd4f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24302/
Test PASSed.

@SparkQA
Copy link

SparkQA commented Dec 10, 2014

Test build #24305 has finished for PR 3661 at commit be42bcf.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24305/
Test PASSed.

def notifyStop() = synchronized {
stopped = true
notifyAll()
def notifyError(e: Throwable) = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor nit, but mind adding : Unit here?

@JoshRosen
Copy link
Contributor

Maybe a naive question, but is there a reason why we can't just use synchronized methods here?

@zsxwing
Copy link
Member Author

zsxwing commented Dec 17, 2014

Maybe a naive question, but is there a reason why we can't just use synchronized methods here?

I wrote the reason in the description: "Used Condition to rewrite ContextWaiter because it provides a convenient API awaitNanos for timeout."

@SparkQA
Copy link

SparkQA commented Dec 17, 2014

Test build #24521 has started for PR 3661 at commit 52247f5.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 17, 2014

Test build #24521 has finished for PR 3661 at commit 52247f5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24521/
Test PASSed.

* Return `true` if it's stopped; or throw the reported error if `notifyError` has been called; or
* `false` if the waiting time detectably elapsed before return from the method.
*/
def waitForStopOrError(timeout: Long = -1): Boolean = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I considered whether this should throw TimeoutException instead of signaling failure via a boolean, but I guess this is sort of like Object.wait() or Condition.wait(), both of which don't throw exceptions.

Also, it looks like the usages of this in StreamingContext support this theory:

  /**
   * Wait for the execution to stop. Any exceptions that occurs during the execution
   * will be thrown in this thread.
   */
  def awaitTermination() {
    waiter.waitForStopOrError()
  }

  /**
   * Wait for the execution to stop. Any exceptions that occurs during the execution
   * will be thrown in this thread.
   * @param timeout time to wait in milliseconds
   */
  def awaitTermination(timeout: Long) {
    waiter.waitForStopOrError(timeout)
  }

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suppose the documentation for the SparkContext.awaitTermination(timeout) could be improved to convey what happens when the timeout occurs...

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think if it's better to return Boolean to indicate if it's timeout? Although it will break the source compatibility.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another option would be to just add a new def isTerminated: Boolean method, which would let users write code like

waiter.awaitTermination(1000)
if (!waiter.isTerminated) {
  throw Exception(...)
}

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed. Done.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about making awaitTermination throw a TimeoutException if timeout? It looks a better API.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@tdas what do you think?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that will break a lot of existing programs for people. A lot of time (even in our own test cases) awaitTermination(timeout) is used for wait for a short period of time before check checking status or something. Currently that times out return quietly. If instead it starts throwing exceptions, then it will completely break some applications. So I strongly advise against changing this behavior.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In hindsight, instead of modeling awaitTermination against Akka ActorSystem's awaitTermination (which return Unit) , I should have modeled it like Java ExecutorService's awaitTermination which returns a Boolean. Now its not possible to change the API without breaking compatiblity. :(

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In hindsight, instead of modeling awaitTermination against Akka ActorSystem's awaitTermination (which return Unit) , I should have modeled it like Java ExecutorService's awaitTermination which returns a Boolean. Now its not possible to change the API without breaking compatiblity. :(

@tdas, Sorry that I forgot to reply you. You said you designed it just like Akka ActorSystem.awaitTermination. But ActorSystem.awaitTermination will throw a TimeoutException in case of timeout.

  /**
   * Block current thread until the system has been shutdown, or the specified
   * timeout has elapsed. This will block until after all on termination
   * callbacks have been run.
   *
   * @throws TimeoutException in case of timeout
   */
  def awaitTermination(timeout: Duration): Unit

@JoshRosen
Copy link
Contributor

This looks good to me.

@SparkQA
Copy link

SparkQA commented Dec 18, 2014

Test build #24568 has started for PR 3661 at commit 15357d2.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 18, 2014

Test build #24568 has finished for PR 3661 at commit 15357d2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean)

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24568/
Test PASSed.

@tdas
Copy link
Contributor

tdas commented Dec 29, 2014

This almost looks good. Can you add a unit test in the StreamingContextSuite to test the new public API isTerminated (with and without error)? You could update the existing unit tests.

@tdas
Copy link
Contributor

tdas commented Dec 29, 2014

Actually, correction. Can you put the public API isTerminated on a different PR? I want to separate the addition to public API (which requires a design discussion) from this bug fix PR. In that case, there is no need to update unit tests, and this PR will be good to go.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24876 has started for PR 3661 at commit 52247f5.

  • This patch merges cleanly.

@zsxwing
Copy link
Member Author

zsxwing commented Dec 30, 2014

I deleted the commit about isTerminated. It will be sent in a different PR.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24876 has finished for PR 3661 at commit 52247f5.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24876/
Test FAILed.

@zsxwing
Copy link
Member Author

zsxwing commented Dec 30, 2014

Jenkins, retest this please.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24881 has started for PR 3661 at commit 52247f5.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Dec 30, 2014

Test build #24881 has finished for PR 3661 at commit 52247f5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24881/
Test PASSed.

@tdas
Copy link
Contributor

tdas commented Dec 30, 2014

Merging this. Thanks!

asfgit pushed a commit that referenced this pull request Dec 30, 2014
…e 'spurious wakeup'

Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout.

Author: zsxwing <zsxwing@gmail.com>

Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits:

52247f5 [zsxwing] Add explicit unit type
be42bcf [zsxwing] Update as per review suggestion
e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'

(cherry picked from commit 6a89782)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
@asfgit asfgit closed this in 6a89782 Dec 30, 2014
asfgit pushed a commit that referenced this pull request Dec 30, 2014
…e 'spurious wakeup'

Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout.

Author: zsxwing <zsxwing@gmail.com>

Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits:

52247f5 [zsxwing] Add explicit unit type
be42bcf [zsxwing] Update as per review suggestion
e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'

(cherry picked from commit 6a89782)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
asfgit pushed a commit that referenced this pull request Dec 30, 2014
…e 'spurious wakeup'

Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout.

Author: zsxwing <zsxwing@gmail.com>

Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits:

52247f5 [zsxwing] Add explicit unit type
be42bcf [zsxwing] Update as per review suggestion
e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'

(cherry picked from commit 6a89782)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
@zsxwing zsxwing deleted the SPARK-4813 branch December 31, 2014 01:31
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants