This sounds like it may be exactly the problem we've been having (and about
which I recently  posted on the user list).

Is there any way of monitoring it's attempts to wait, giving up, and trying
another level?

In general, I'm trying to figure out why we can have repeated identical
jobs, the first of which will have all PROCESS_LOCAL, and the next will
have 95% PROCESS_LOCAL, and 5% ANY.


On Thu, Nov 13, 2014 at 2:20 PM, Kay Ousterhout <k...@eecs.berkeley.edu>
wrote:

> Hi,
>
> Shivaram and I stumbled across this problem a few weeks ago, and AFAIK
> there is no nice solution.  We worked around it by avoiding jobs with tasks
> that have tasks with two locality levels.
>
> To fix this problem, we really need to fix the underlying problem in the
> scheduling code, which currently tries to schedule all tasks at the minimum
> locality of any of the tasks in the job.  Unfortunately, this involves
> adding a bunch of complexity to the scheduling code.
>
> Patrick had previously convinced us that we were the only ones running into
> this problem, so it wasn't worth fixing (especially because we found an OK
> workaround for our case).  It would be useful to hear if any other folks
> have run into this problem -- it sounds like maybe we should go ahead and
> fix the scheduling code.
>
> I've filed a JIRA to track this:
> https://issues.apache.org/jira/browse/SPARK-4383
>
> -Kay
>
> On Wed, Nov 12, 2014 at 11:55 PM, MaChong <machon...@sina.com> wrote:
>
> > Hi,
> >
> > We are running a time sensitive application with 70 partition and 800MB
> > each parition size. The application first load data from database in
> > different cluster, then apply a filter, cache the filted data, then
> apply a
> > map and a reduce, finally collect results.
> > The application will be finished in 20 seconds if we set
> > spark.locality.wait to a large value (30 minutes). And it will use 100
> > seconds, if we set spark.locality.wait a small value(less than 10
> seconds)
> > We have analysed the driver log and found lot of NODE_LOCAL and
> RACK_LOCAL
> > level tasks, normally a PROCESS_LOCAL task only takes 15 seconds, but
> > NODE_LOCAL or RACK_LOCAL tasks will take 70 seconds.
> >
> > So I think we'd better set spark.locality.wait to a large value(30
> > minutes), until we meet this problem:
> >
> > Now our application will load data from hdfs in the same spark cluster,
> it
> > will get NODE_LOCAL and RACK_LOCAL level tasks during loading stage, if
> the
> > tasks in loading stage have same locality level, ether NODE_LOCAL or
> > RACK_LOCAL it works fine.
> > But if the tasks in loading stage get mixed locality level, such as 3
> > NODE_LOCAL tasks, and 2 RACK_LOCAL tasks, then the TaskSetManager of
> > loading stage will submit the 3 NODE_LOCAL tasks as soon as resources
> were
> > offered, then wait for spark.locality.wait.node, which was setted to 30
> > minutes, the 2 RACK_LOCAL tasks will wait 30 minutes even though
> resources
> > are avaliable.
> >
> >
> > Does any one have met this problem? Do you have a nice solution?
> >
> >
> > Thanks
> >
> >
> >
> >
> > Ma chong
> >
>



-- 
Nathan Kronenfeld
Senior Visualization Developer
Oculus Info Inc
2 Berkeley Street, Suite 600,
Toronto, Ontario M5A 4J5
Phone:  +1-416-203-3003 x 238
Email:  nkronenf...@oculusinfo.com

Reply via email to