[ 
https://issues.apache.org/jira/browse/LIVY-459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16793757#comment-16793757
 ] 

Thoralf Gutierrez edited comment on LIVY-459 at 3/15/19 4:32 PM:
-----------------------------------------------------------------

We found the issue, here's what happens:

When creating the python client (client = HttpClient(...)) the interpreter gets 
started. If you then try to submit a job (client.submit(...).result()), Livy 
will first check whether your PySpark interpreter is up or not. If it is not up 
yet, it will send your request to the java interpreter path and you end up with 
a KryoException

[https://github.com/apache/incubator-livy/blob/292896bacfd873ce69c682ba584a03ef8455bd52/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala#L94-L102]

On the other hand, if you wait for your Livy session to become "idle" before 
you send your client.submit, everything will work as expected.

So one option would be to make Livy return an error if you are trying to submit 
a job while the pyspark interpreter isn't ready yet (instead of taking the java 
path), to indicate to the client that it should retry in a couple of seconds 
and implement that retry loop in the client.

In the meantime though, you can change your python client to check with the 
session is still starting or idle (with the session state API) before 
submitting your job.


was (Author: thorgutierrez):
We found the issue, here's what happens:

When creating the python client (client = HttpClient(...)) the interpreter gets 
started. If you then try to submit a job (client.submit(...).result()), Livy 
will first check whether your PySpark interpreter is up or not. If it is not up 
yet, it will send your request to the java interpreter path and you end up with 
a KryoException

[https://github.com/apache/incubator-livy/blob/292896bacfd873ce69c682ba584a03ef8455bd52/repl/src/main/scala/org/apache/livy/repl/ReplDriver.scala#L94-L102]

On the other hand, if you wait for your Livy session to become "idle" before 
you send your client.submit, everything will work as expected.

So one option would be to make Livy return an error if you are trying to submit 
a job while the pyspark interpreter isn't ready yet (instead of taking the java 
path), to indicate to the client that it should retry in a couple of seconds 
and implement that retry loop in the client.

In the meantime though, you can perform that check in your python client by 
hitting the session state API.

> First submission via livy-python-api fails with KryoException, subsequent 
> retries succeed
> -----------------------------------------------------------------------------------------
>
>                 Key: LIVY-459
>                 URL: https://issues.apache.org/jira/browse/LIVY-459
>             Project: Livy
>          Issue Type: Bug
>    Affects Versions: 0.5.0, 0.6.0
>         Environment: Red Hat Enterprise Linux Server release 7.3 (Maipo)
> java version "1.8.0_141"
>            Reporter: Matthias Wolf
>            Priority: Major
>
> I have the following code:
> {code}
> from livy.client import HttpClient
> def foobar(ctx):
>     return ctx.sc.parallelize(range(101)).mean()
> client = HttpClient('http://r2i0n33:8998')
> try:
>     print(client.submit(foobar).result())
> except Exception as e:
>     print(e)
> print(client.submit(foobar).result())
> client.stop(True)
> {code}
> failing with the following error:
> {code:java}
> org.apache.livy.shaded.kryo.kryo.KryoException: Encountered unregistered 
> class ID: 510
> org.apache.livy.shaded.kryo.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:119)
> org.apache.livy.shaded.kryo.kryo.Kryo.readClass(Kryo.java:656)
> org.apache.livy.shaded.kryo.kryo.Kryo.readClassAndObject(Kryo.java:767)
> org.apache.livy.client.common.Serializer.deserialize(Serializer.java:63)
> org.apache.livy.rsc.driver.BypassJob.call(BypassJob.java:39)
> org.apache.livy.rsc.driver.BypassJob.call(BypassJob.java:27)
> org.apache.livy.rsc.driver.JobWrapper.call(JobWrapper.java:57)
> org.apache.livy.rsc.driver.BypassJobWrapper.call(BypassJobWrapper.java:42)
> org.apache.livy.rsc.driver.BypassJobWrapper.call(BypassJobWrapper.java:27)
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> java.lang.Thread.run(Thread.java:748)
> 50.0
> {code}
> Is this to be expected? I would have thought that the first submission should 
> work already. I did not find this behavior online/in the mailing list 
> archives. Any pointers how to resolve this?
> This happens in both the livy-0.5.0 binaries and the github master, on RedHat 
> EL 7.3.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to