Skip to content

Commit

Permalink
[LIVY-519][TEST] Fix travis failed on should kill yarn app
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

Fix travis failed on "should kill yarn app"

The cause of failed is as follows:
1. When create SparkYarnApp, the yarnAppMonitorThread will be created, which change app state to Failed. Because before recent commit apache@a90f4fa, the pair <RUNNING: getYarnApplicationState, SUCCEEDED: getFinalApplicationStatus> which was mocked in test, but was not defined in mapYarnState, so the state of app will be changed to failed.

2. Then the test kills app, which will call killApplication when the app is running. However the app has been changed to failed in step 1, so killApplication won't be called, and verify(mockYarnClient).killApplication(appId) failed.

3. So if yarnAppMonitorThread changes app state before main thread kills app, the test will failed. If not, the test will succeed.

4. Though the recent commit apache@a90f4fa fixed the bug accidentally, it is necessary to ensure the app is running before kill app.

## How was this patch tested?

Existed UT and IT.

Author: runzhiwang <[email protected]>

Closes apache#221 from runzhiwang/LIVY-519.
  • Loading branch information
runzhiwang authored and jerryshao committed Sep 3, 2019
1 parent d0d8028 commit 830d740
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -209,7 +209,8 @@ class SparkYarnApp private[utils] (
.getOrElse(IndexedSeq.empty)
}

private def isRunning: Boolean = {
// Exposed for unit test.
private[utils] def isRunning: Boolean = {
state != SparkApp.State.FAILED && state != SparkApp.State.FINISHED &&
state != SparkApp.State.KILLED
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.mockito.stubbing.Answer
import org.scalatest.FunSpec
import org.scalatest.mock.MockitoSugar.mock

import org.apache.livy.{LivyBaseUnitTestSuite, LivyConf}
import org.apache.livy.{LivyBaseUnitTestSuite, LivyConf, Utils}
import org.apache.livy.utils.SparkApp._

class SparkYarnAppSpec extends FunSpec with LivyBaseUnitTestSuite {
Expand Down Expand Up @@ -145,6 +145,7 @@ class SparkYarnAppSpec extends FunSpec with LivyBaseUnitTestSuite {
when(mockYarnClient.getApplicationReport(appId)).thenReturn(mockAppReport)

val app = new SparkYarnApp(appTag, appIdOption, None, None, livyConf, mockYarnClient)
Utils.waitUntil({ () => app.isRunning }, Duration(10, TimeUnit.SECONDS))
cleanupThread(app.yarnAppMonitorThread) {
app.kill()
appKilled = true
Expand Down

0 comments on commit 830d740

Please sign in to comment.