ok. but i would beware of possible leaks associated with tons of session recycling. at least i did not stress it enough even in standalone mode to assert meaningful leakage. I only tried for concurrency problems.
On Wed, Nov 20, 2013 at 1:56 PM, Mingyu Kim <[email protected]> wrote: > Yea, we want multiple contexts for isolation of imported jars. We’d like > our users to submit jobs with their own versions of helper libraries (ones > that they write) and updating jars at runtime can break a lot of systems. > > Okay, so we’d need to stick with the standalone mode if we were to use > multiple SparkContexts in one JVM. Thanks! > > Mingyu > > From: Dmitriy Lyubimov <[email protected]> > Reply-To: "[email protected]" < > [email protected]> > Date: Wednesday, November 20, 2013 at 1:42 PM > To: "[email protected]" <[email protected]> > Subject: Re: Multiple SparkContexts in one JVM > > Oh. i suppose if you mean you may install backend closures at will -- yes, > this would present problem in the sense that since session is already set > up, one cannot update its backend closures. > > Alas, fwiw spark-0.8 doesn't make any claims officially as to concurrency > guarantees of multiple contexts in the same JVM. In practice it has always > worked for me in standalone mode and never worked in Mesos mode (mesos > backend deadlocks). > > > On Wed, Nov 20, 2013 at 1:39 PM, Dmitriy Lyubimov <[email protected]>wrote: > >> >> >> >> On Wed, Nov 20, 2013 at 12:56 PM, Matt Cheah <[email protected]> wrote: >> >>> Our use case is trying to isolate the classes in shipped jars that are >>> available to different users of the Spark context. >>> >>> We have multiple users that can run against our Web Server JVM which >>> will host a Spark Context. These users can compile their own jars and query >>> the SparkContext to add them (addJar(…)). However, concurrent users could >>> add jars that could possibly cause class namespace collisions and race >>> conditions. In addition, one user may submit one job to the SparkContext >>> with a jar, then finish that job and wish to update that jar on a >>> subsequent job. That's why we want to create a SparkContext per >>> compute-session to sandbox the jars in each user's job. >>> >>> Is this a use case that could be done by only using one SparkContext in >>> the JVM? >>> >> >> Technically, i don't see why not -- at least that's the claim in 0.8 and >> that's the update i've made to our system in the last release as we >> migrated to 0.8. Although we haven't put the context-concurrent request >> patch in production yet, our QA process has not found any issues with that >> yet. >> >> Multiple namespaces seems to me to be something irrelevant to reentrancy >> issue (it can't either help or harm reentrancy capability itself) >> >> >>> -Matt Cheah >>> >>> From: Dmitriy Lyubimov <[email protected]> >>> Reply-To: "[email protected]" < >>> [email protected]> >>> Date: Wednesday, November 20, 2013 12:39 PM >>> To: "[email protected]" <[email protected]> >>> Subject: Re: Multiple SparkContextx in one JVM >>> >>> As far as i can tell, the mesos back end would still not work correctly >>> with multiple SparkContexts. >>> >>> However, if you are just after spark query concurrency, spark 0.8 seems >>> to be supporting concurrent (reentrant) requests to the same session >>> (SparkContext). One should also be able to use FAIR scheduler in this case >>> it seems (at least that's what i request). So i just sempahore the same >>> context while keeping pool of SparkContext's at 1. >>> >>> See the doc folder for details. >>> >>> >>> >>> On Wed, Nov 20, 2013 at 3:26 AM, Mingyu Kim <[email protected]> wrote: >>> >>>> Hi all, >>>> >>>> I’ve been searching to find out the current status of the multiple >>>> SparkContext support in one JVM. I found >>>> https://groups.google.com/forum/#!topic/spark-developers/GLx8yunSj0A >>>> and https://groups.google.com/forum/#!topic/spark-users/cOYP96I668I. >>>> According to the threads, I should be able to create multiple SparkContexts >>>> by setting the port to “0” every time. However, Matei mentioned that >>>> SparkEnv should be part of TaskContext rather than being thread local. Does >>>> this cause any problem with running multiple SparkContexts in one JVM right >>>> now or is this just a clean-up that needs to be done eventually? I’m >>>> wondering if Spark supports multiple SparkContexts as is right now or if >>>> there is anything I should be careful about when creating multiple >>>> SparkContexts. >>>> >>>> Thanks! >>>> >>>> Mingyu >>>> >>> >>> >> >
