You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@livy.apache.org by "Jeff Xu (Jira)" <ji...@apache.org> on 2022/10/14 17:53:00 UTC
[jira] [Created] (LIVY-896) Livy could intermittently returns batch as SUCCEED even Spark on Yarn actually fails
Jeff Xu created LIVY-896:
----------------------------
Summary: Livy could intermittently returns batch as SUCCEED even Spark on Yarn actually fails
Key: LIVY-896
URL: https://issues.apache.org/jira/browse/LIVY-896
Project: Livy
Issue Type: Bug
Components: Server
Reporter: Jeff Xu
Attachments: livy_batch.py, livy_client.py
*Summary:*
* I ran into this issue using AWS EMR.
* Frequency of the issue varies. On one EMR cluster, I typically see ~10-20% chance of hitting the issue. But on another EMR cluster, the chance is ~1%. I suspect the chance depends on how busy AWS hardware actually was (my EMR likely share hardware resources with other AWS tenants).
* I believe that I have identify the root cause in Livy source code (refer to a later section).
*How to reproduce:*
* An EMR with Spark, Yarn and Livy configured.
* Use the attached livy_batch.py to trigger a Livy batch by using livy python client (0.8.0). See attached livy_client.py.
* Repeat the testing and you should see when the issue happens, even though the spark program errors out, Livy still reports the batch as SUCCEED.
*Livy log for a good case when Livy returns batch as DEAD (expected behavior):*
22/10/14 02:46:22 INFO BatchSessionManager: Registered new session 1
22/10/14 02:46:42 DEBUG BatchSession: BatchSession 1 state changed from STARTING to RUNNING
22/10/14 02:46:43 WARN BatchSession$: spark-submit exited with code 1
22/10/14 02:46:47 DEBUG BatchSession: BatchSession 1 state changed from RUNNING to FINISHED
22/10/14 02:46:47 DEBUG BatchSession: BatchSession 1 state changed from FINISHED to FAILED
*Livy log for bad case when Livy returns batch as SUCCEED (bug):*
22/10/14 02:47:40 INFO BatchSessionManager: Registered new session 3
22/10/14 02:48:00 DEBUG BatchSession: BatchSession 3 state changed from STARTING to FINISHED
22/10/14 02:48:01 WARN BatchSession$: spark-submit exited with code 1
*Root cause analysis:*
* I think the bug is in the YarnAppMonitorThread of Livy server.
* The bug is in this section: [https://github.com/apache/incubator-livy/blob/v0.7.1-incubating-rc1/server/src/main/scala/org/apache/livy/utils/SparkYarnApp.scala#L283-L299]
* When the right timing happens,
** Yarn sees the application completed
** But spark-submit process is still running
* So line 286 returns app report saying the application completes successfully (Yarn has not context of the Spark application succeeds or not).
* Line 288 set the Livy session as succeed.
* Line 293 returns false as the spark-submit is still running.
* So we hit the bug.
Even without hitting the timing condition, the code logic itself is {*}still incorrect{*}.
If you take a look at the log from a "good" case, the session state was updated twice: FINISHED, then FAILED. If a client query arrives on the perfect timing, the livy server could can still return a wrong state.
{noformat}
22/10/14 02:46:47 DEBUG BatchSession: BatchSession 1 state changed from RUNNING to FINISHED
22/10/14 02:46:47 DEBUG BatchSession: BatchSession 1 state changed from FINISHED to FAILED{noformat}
I hope we can work together to have the issue addressed ASAP as the bug hit our production code pretty bad. I think the right code logic should be:
# read the spark-submit process's state, if still running, do nothing
# If the spark-submit process finishes, read Yarn report, and determines the actual application finish state in a single shot.
# Update the session state in a single step.
At the same time, I will see if I can create a PR with suggested fix soon. The challenge on my side is that it's almost impossible for me to swap a few jars from open-source code base on AWS EMR (not compatible with EMR runtime).
Thank you, Livy team!
Regards,
Jeff Xu, a Workday engineer
--
This message was sent by Atlassian Jira
(v8.20.10#820010)