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

Saisai Shao commented on SPARK-2780:
------------------------------------

Hi TD, I think the fair scheduler setting can be worked even in different 
thread. I think I fixed this issue long long ago in 
(https://github.com/mesos/spark/pull/937). The local property can be gotten in 
child thread, since this thread local variable is heritable.

I think the reason why it is not worked is that we set local property after the 
child thread is started:

{code}
ssc.sparkContext.setLocalProperty(...)
{code}

Actually some children threads are already started when StreamingContext is 
created, like job executor thread pool. so we need to create SparkContext and 
set local property before creating StreamingContext to let local property be 
gotten by child thread, like:

{code}
sc = new SparkContext(...)
sc.setLocalProperty(...)
ssc = new StreamingContext(sc, ...)
{code}

I think in this way local property can be effected, though a little strange 
from user's point of view. Maybe we should figure out a better way to set local 
property in StreamingContext.

> Create a StreamingContext.setLocalProperty for setting local property of jobs 
> launched by streaming
> ---------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-2780
>                 URL: https://issues.apache.org/jira/browse/SPARK-2780
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.0.0, 1.1.0
>            Reporter: Tathagata Das
>            Priority: Minor
>
> SparkContext.setLocalProperty makes all Spark jobs submitted using
> the current thread belong to the set pool. However, in Spark
> Streaming, all the jobs are actually launched in the background from a
> different thread. So this setting does not work. 
> Currently, there is a
> work around. If you are doing any kind of output operations on
> DStreams, like DStream.foreachRDD(), you can set the property inside
> that
> dstream.foreachRDD(rdd =>
>    rdd.sparkContext.setLocalProperty(...)
> )



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to