incubator-livy
incubator-livy copied to clipboard
[LIVY-896] Livy not capture spark-submit error exit if timing is right
What changes were proposed in this pull request?
Proposed code fix for https://issues.apache.org/jira/browse/LIVY-896.
How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests
Two new unit tests are included in the PR. Outputs of the new unit tests are attached as unit-tests.txt.
System tests
I run system tests manually to verify the fix on an EMR in AWS.
- Install open source Livy 0.7.1 and configured it running on port 8999.
- Upload my PySpark program run_sql.py to the cluster's HDFS (see the artifact below)
- Loop my test_livy.py (see the artifact below) for 10 times. I hit the issue 5 out of 10 tries (Livy reported SessionState.SUCCESS even though spark-submit failed). See reproduced.txt for details.
- Replaced livy-server jar with the fixed version.
- Loop my test_livy.py for 100 times. I never hit the issue again. See fixed.txt for details.
- I also quickly verified that Livy Session does not have the issue. See livy_session.txt for details.
Artifact: test_livy.py (the verification program)
#!/usr/bin/env python3
from livy import LivyBatch
import sys
import logging
logging.basicConfig(format='%(asctime)s %(levelname)s %(message)s', stream=sys.stderr, level=logging.INFO)
if __name__ == "__main__":
batch = LivyBatch.create(
url="http://ip-100-64-129-199.us-west-2.compute.internal:8999",
file="/tmp/run_sql.py", args=["-s", "select * from abc"],
)
logging.info(f"batch id={batch.batch_id} created ...")
batch.wait()
logging.info(f"batch id={batch.batch_id}, state={batch.state}")
Artifact: run_sql.py (the Spark program to run a given SQL)
from pyspark.sql import SparkSession
import sys
import argparse
if __name__ == "__main__":
parser = argparse.ArgumentParser(
formatter_class = argparse.ArgumentDefaultsHelpFormatter,
)
parser.add_argument("-s", action="store", dest="sql")
args = parser.parse_args()
spark = SparkSession.builder.\
appName("PySpark SparkSQL").\
enableHiveSupport().\
config("spark.ui.enabled", "false").\
getOrCreate()
try:
spark.sql(args.sql).show()
finally:
spark.stop()
Artifact: test_session.py (verify Livy session does not have the issue)
#!/usr/bin/env python3
from livy import LivySession, SessionKind
import sys
import logging
logging.basicConfig(format='%(asctime)s %(levelname)s %(message)s', stream=sys.stderr, level=logging.INFO)
if __name__ == "__main__":
sess = LivySession.create(
url="http://ip-100-64-129-199.us-west-2.compute.internal:8999",
kind=SessionKind.SQL)
logging.info(f"session id={sess.session_id} created ...")
sess.wait()
logging.info(f"session id={sess.session_id} is ready")
try:
sess.download_sql("SELECT * from xxyyzz")
except Exception as e:
logging.info(str(e))
Please review https://livy.incubator.apache.org/community/ before opening a pull request.
@jerryshao @alex-the-man @ajbozarth Would you please take a look at the build failures? The errors don't seem to be related to my changes. Also I can see the same error started occurring a while ago.
Thanks, Jeff
@jeff-xu-z would you please rebase to latest master to fix the CI?
Hi @leesf:
This is what I see in my LIVY-896. I should already be on top of latest master commit (4d8a912699683b973eee76d4e91447d769a0cb0d).
I just tried "git checkout master && git pull". I didn't see any new changes from master branch.
commit fb5f9d08d0a6dc84a62658269bcbb3522d37a0ac (HEAD -> LIVY-896, origin/LIVY-896)
Author: Jeff Xu <[email protected]>
Date: Fri Oct 14 12:53:10 2022 -0700
[LIVY-896] Livy not capture spark-submit error exit if timing is right
commit 4d8a912699683b973eee76d4e91447d769a0cb0d (origin/master, origin/HEAD, master)
Author: Marco Gaido <[email protected]>
Date: Fri Aug 14 17:25:54 2020 -0700
Thanks, Jeff
Hi @leesf:
This is what I see in my LIVY-896. I should already be on top of latest master commit (4d8a912).
I just tried "git checkout master && git pull". I didn't see any new changes from master branch.
commit fb5f9d08d0a6dc84a62658269bcbb3522d37a0ac (HEAD -> LIVY-896, origin/LIVY-896) Author: Jeff Xu <[email protected]> Date: Fri Oct 14 12:53:10 2022 -0700 [LIVY-896] Livy not capture spark-submit error exit if timing is right commit 4d8a912699683b973eee76d4e91447d769a0cb0d (origin/master, origin/HEAD, master) Author: Marco Gaido <[email protected]> Date: Fri Aug 14 17:25:54 2020 -0700Thanks, Jeff
@jeff-xu-z something must be wrong, you would see that I did push a fix on the master branch. maybe you need git checkout master && git pull upstream master the apache/livy repo should be upstream instead of origin in your config?
@jeff-xu-z pls check the travis CI failure.
@leesf Pushed an updated commit. The build failure looks to be a scala style check failure.
[INFO] --- scalastyle-maven-plugin:0.8.0:check (default) @ livy-server ---
error file=/home/travis/build/apache/incubator-livy/server/src/test/scala/org/apache/livy/utils/SparkYarnAppSpec.scala message=File line length exceeds 100 characters line=272
Codecov Report
Merging #358 (aeba182) into master (1954868) will increase coverage by
0.10%. The diff coverage is64.28%.
@@ Coverage Diff @@
## master #358 +/- ##
============================================
+ Coverage 68.35% 68.45% +0.10%
- Complexity 838 842 +4
============================================
Files 103 103
Lines 5940 5948 +8
Branches 898 899 +1
============================================
+ Hits 4060 4072 +12
+ Misses 1318 1314 -4
Partials 562 562
| Impacted Files | Coverage Δ | |
|---|---|---|
| ...ain/scala/org/apache/livy/utils/SparkYarnApp.scala | 72.61% <64.28%> (-1.14%) |
:arrow_down: |
| ...ain/java/org/apache/livy/rsc/driver/RSCDriver.java | 80.00% <0.00%> (+1.25%) |
:arrow_up: |
| ...main/scala/org/apache/livy/server/LivyServer.scala | 32.58% <0.00%> (+1.78%) |
:arrow_up: |
| ...c/main/scala/org/apache/livy/repl/ReplDriver.scala | 38.46% <0.00%> (+2.56%) |
:arrow_up: |
:mega: We’re building smart automated test selection to slash your CI/CD build times. Learn more
@leesf Thank you for accepted & merged the PR! My 1st open source contribution. 👯