You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Xingbo Jiang <ji...@gmail.com> on 2020/02/28 19:21:20 UTC

[DISCUSS] Remove multiple workers on the same host support from Standalone backend

Hi all,

Based on my experience, there is no scenario that necessarily requires
deploying multiple Workers on the same node with Standalone backend. A
worker should book all the resources reserved to Spark on the host it is
launched, then it can allocate those resources to one or more executors
launched by this worker. Since each executor runs in a separated JVM, we
can limit the memory of each executor to avoid long GC pause.

The remaining concern is the local-cluster mode is implemented by launching
multiple workers on the local host, we might need to re-implement
LocalSparkCluster to launch only one Worker and multiple executors. It
should be fine because local-cluster mode is only used in running Spark
unit test cases, thus end users should not be affected by this change.

Removing multiple workers on the same host support could simplify the
deploy model of Standalone backend, and also reduce the burden to support
legacy deploy pattern in the future feature developments. (There is an
example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
designed a complex approach to coordinate resource requirements from
different workers launched on the same host).

The proposal is to update the document to deprecate the support of system
environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the support
in the next major version (Spark 3.1).

Please kindly let me know if you have use cases relying on this feature.

Thanks!

Xingbo

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Sean Owen <sr...@gmail.com>.
On Sat, Mar 14, 2020 at 5:56 PM Andrew Melo <an...@gmail.com> wrote:
> Sorry, I'm from a completely different field, so I've inherited a completely different vocabulary. So thanks for bearing with me :)
>
> I think from reading your response, maybe the confusion is that HTCondor is a completely different resource acquisition model than what industry is familiar with. Unlike AWS that gives you a whole VM or k8s that gives you a whole container, condor (and most other batch schedulers) split up a single bare machine that your job shares with whatever else is on that machine. You don't get your own machine or even the illusion you have your own machine (via containerization).
>
> Using these schedulers it's not that you ask for N workers when there's only M machines, you request N x 8core slots when there are M cores available, and the scheduler packs them wherever there's free resources.

Actually, that's exactly what a Spark standalone worker or YARN
NodeManager does. It allocates resources on a shared machine, without
virtualization. If there were Spark <> HTCondor integration, you'd
really just submit apps to the HTCondor cluster and let it allocate
_executors_ for the app for you.

Indeed you would not generally expect a resource manager to guarantee
where the resources come from. So it's possible and normal to have
multiple executors allocated by the resource manager on one machine,
for the same app.

It's not so normal to allocate multiple workers (resource manager
daemons) on a set of physical resources; it needlessly chops them up,
or even, risks them both thinking they're in charge of the same
resources. So, in Spark standalone where you control where workers
run, you wouldn't normally run multiple ones per machine. You'd let
one manage whatever resources the Spark cluster should take on the
hardware. Likewise YARN has one NodeManager per machine.

Here, you have the extra step here of allocating a resource manager
(Spark standalone) within your resource manager (HTCondor) because
there is no direct integration. And I think that's the issue. Resource
manager HTCondor isn't necessarily allocating resources in a way that
makes sense for a second-level resource manager.


> If you're talking about the 2nd half, let's say I'm running two pyspark notebooks connected to the system above, and batch scheduler gives each of them 2 cores of slaves. Each notebook will have their own set (which I called a pool earlier) of slaves, so when you're working in one notebook, the other notebook of slaves is idle. My comment was about the resources being idle and the desire to increase utillzation.

I think you are saying each job spins up a whole new Spark cluster,
and every Spark cluster runs just one app. That's not crazy at all,
though, normally you would also have the possibility of one cluster
running N apps of course, and better sharing its resources. But it
sounds like it's the way you have to do it.

Well I can see some possible outcomes:

1) Can you not use HTCondor? allocate a long-lived Spark standalone
cluster instead on resources managed only by the Spark cluster, and
submit apps to it. The price is no reuse of resources with other
non-Spark applications
2) Can HTCondor be convinced to allocate chunks of resources on
distinct machines? that'd do it too
3) HTCondor can't be convinced to do any isolation of the processes
themselves right? because if the workers aren't on the same 'virtual'
machine or space then it all works out, which is why all this works
fine on K8S.
4) .. just keep this functionality in Spark as a sort of generic
resource manager bridge for cases like this. We may have identified
the perhaps niche but real use case for it beyond testing

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org


Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Andrew Melo <an...@gmail.com>.
Hi Sean

On Fri, Mar 13, 2020 at 6:46 PM Sean Owen <sr...@gmail.com> wrote:

> Do you really need a new cluster per user? and if so, why specify N
> workers > M machines? I am not seeing a need for that. I don't even
> think 2 workers on the same host makes sense, as they are both
> managing the same resources; it only exists for test purposes AFAICT.
>

Sorry, I'm from a completely different field, so I've inherited a
completely different vocabulary. So thanks for bearing with me :)

I think from reading your response, maybe the confusion is that HTCondor is
a completely different resource acquisition model than what industry is
familiar with. Unlike AWS that gives you a whole VM or k8s that gives you a
whole container, condor (and most other batch schedulers) split up a single
bare machine that your job shares with whatever else is on that machine.
You don't get your own machine or even the illusion you have your own
machine (via containerization).

Using these schedulers it's not that you ask for N workers when there's
only M machines, you request N x 8core slots when there are M cores
available, and the scheduler packs them wherever there's free resources.

> What you are trying to do sounds like one cluster, not many. JVMs

> can't be shared across users; JVM = executor. But that's a good thing,
> or else there would be all kinds of collisions.


> What pools are you referring to?


If you're talking about the 2nd half, let's say I'm running two pyspark
notebooks connected to the system above, and batch scheduler gives each of
them 2 cores of slaves. Each notebook will have their own set (which I
called a pool earlier) of slaves, so when you're working in one notebook,
the other notebook of slaves is idle. My comment was about the resources
being idle and the desire to increase utillzation.

Thanks
Andrew

Sean
>
> On Fri, Mar 13, 2020 at 6:33 PM Andrew Melo <an...@gmail.com> wrote:
> >
> > Hi Xingbo, Sean,
> >
> > On Fri, Mar 13, 2020 at 12:31 PM Xingbo Jiang <ji...@gmail.com>
> wrote:
> >>
> >> Andrew, could you provide more context of your use case please? Is it
> like you deploy homogeneous containers on hosts with available resources,
> and each container launches one worker? Or you deploy workers directly on
> hosts thus you could have multiple workers from the same application on the
> same host?
> >
> >
> > Sure, I describe a bit more detail about the actual workload below [*],
> but the short version is that our computing resources/infrastructure are
> all built around batch submission into (usually) the HTCondor scheduler,
> and we've got a PoC using pyspark to replace the interactive portion of
> data analysis. To run pyspark on our main resources, we use some scripts
> around the standalone mode to spin up N slaves per-user**, which may or may
> not end up on the same host. I understood Xingbo's original mail to mean
> that wouldn't be allowed in the future, but from Sean's response, it seems
> like I'm incorrect.
> >
> > That being said, our use-case is very bursty, and it would be very good
> if there was a way we could have one pool of JVMs that could be shared
> between N different concurrent users instead of having N different pools of
> JVMs that each serve one person. We're already resource constrained, and
> we're expecting our data rates to increase 10x in 2026, so the less idle
> CPU, the better for us.
> >
> > Andrew
> >
> > * I work for one of the LHC experiments at CERN (https://cms.cern/) and
> there's two main "phases" of our data pipeline: production and analysis.
> The analysis half is currently implemented by having users writing some
> software, splitting the input dataset(s) into N parts and then submitting
> those jobs to the batch system (combining the outputs in a manual
> postprocessing step). In terms of scale, there are currently ~100 users
> running ~900 tasks over ~50k cpus. The use case relevant to this context is
> the terminal analysis phase which involves calculating some additional
> columns, applying calibrations, filtering out the 'interesting' events and
> extracting histograms describing those events. Conceptually, it's an
> iterative process of "extract plots, look at plots, change parameters", but
> running in a batch system means the latency is bad, so it can take a long
> time to converge to the right set of params.
> >
> > ** though we have much smaller, dedicated k8s/mesos/yarn clusters we use
> for prototyping
> >
> >>
> >> Thanks,
> >>
> >> Xingbo
> >>
> >> On Fri, Mar 13, 2020 at 10:23 AM Sean Owen <sr...@gmail.com> wrote:
> >>>
> >>> You have multiple workers in one Spark (standalone) app? this wouldn't
> >>> prevent N apps from each having a worker on a machine.
> >>>
> >>> On Fri, Mar 13, 2020 at 11:51 AM Andrew Melo <an...@gmail.com>
> wrote:
> >>> >
> >>> > Hello,
> >>> >
> >>> > On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com>
> wrote:
> >>> >>
> >>> >> Hi all,
> >>> >>
> >>> >> Based on my experience, there is no scenario that necessarily
> requires deploying multiple Workers on the same node with Standalone
> backend. A worker should book all the resources reserved to Spark on the
> host it is launched, then it can allocate those resources to one or more
> executors launched by this worker. Since each executor runs in a separated
> JVM, we can limit the memory of each executor to avoid long GC pause.
> >>> >>
> >>> >> The remaining concern is the local-cluster mode is implemented by
> launching multiple workers on the local host, we might need to re-implement
> LocalSparkCluster to launch only one Worker and multiple executors. It
> should be fine because local-cluster mode is only used in running Spark
> unit test cases, thus end users should not be affected by this change.
> >>> >>
> >>> >> Removing multiple workers on the same host support could simplify
> the deploy model of Standalone backend, and also reduce the burden to
> support legacy deploy pattern in the future feature developments. (There is
> an example in https://issues.apache.org/jira/browse/SPARK-27371 , where
> we designed a complex approach to coordinate resource requirements from
> different workers launched on the same host).
> >>> >>
> >>> >> The proposal is to update the document to deprecate the support of
> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
> support in the next major version (Spark 3.1).
> >>> >>
> >>> >> Please kindly let me know if you have use cases relying on this
> feature.
> >>> >
> >>> >
> >>> > When deploying spark on batch systems (by wrapping the standalone
> deployment in scripts that can be consumed by the batch scheduler), we
> typically end up with >1 worker per host. If I understand correctly, this
> proposal would make our use case unsupported.
> >>> >
> >>> > Thanks,
> >>> > Andrew
> >>> >
> >>> >
> >>> >
> >>> >>
> >>> >> Thanks!
> >>> >>
> >>> >> Xingbo
> >>> >
> >>> > --
> >>> > It's dark in this basement.
> >>>
> >>> ---------------------------------------------------------------------
> >>> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org
> >>>
>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Sean Owen <sr...@gmail.com>.
Do you really need a new cluster per user? and if so, why specify N
workers > M machines? I am not seeing a need for that. I don't even
think 2 workers on the same host makes sense, as they are both
managing the same resources; it only exists for test purposes AFAICT.

What you are trying to do sounds like one cluster, not many. JVMs
can't be shared across users; JVM = executor. But that's a good thing,
or else there would be all kinds of collisions.

What pools are you referring to?

Sean

On Fri, Mar 13, 2020 at 6:33 PM Andrew Melo <an...@gmail.com> wrote:
>
> Hi Xingbo, Sean,
>
> On Fri, Mar 13, 2020 at 12:31 PM Xingbo Jiang <ji...@gmail.com> wrote:
>>
>> Andrew, could you provide more context of your use case please? Is it like you deploy homogeneous containers on hosts with available resources, and each container launches one worker? Or you deploy workers directly on hosts thus you could have multiple workers from the same application on the same host?
>
>
> Sure, I describe a bit more detail about the actual workload below [*], but the short version is that our computing resources/infrastructure are all built around batch submission into (usually) the HTCondor scheduler, and we've got a PoC using pyspark to replace the interactive portion of data analysis. To run pyspark on our main resources, we use some scripts around the standalone mode to spin up N slaves per-user**, which may or may not end up on the same host. I understood Xingbo's original mail to mean that wouldn't be allowed in the future, but from Sean's response, it seems like I'm incorrect.
>
> That being said, our use-case is very bursty, and it would be very good if there was a way we could have one pool of JVMs that could be shared between N different concurrent users instead of having N different pools of JVMs that each serve one person. We're already resource constrained, and we're expecting our data rates to increase 10x in 2026, so the less idle CPU, the better for us.
>
> Andrew
>
> * I work for one of the LHC experiments at CERN (https://cms.cern/) and there's two main "phases" of our data pipeline: production and analysis. The analysis half is currently implemented by having users writing some software, splitting the input dataset(s) into N parts and then submitting those jobs to the batch system (combining the outputs in a manual postprocessing step). In terms of scale, there are currently ~100 users running ~900 tasks over ~50k cpus. The use case relevant to this context is the terminal analysis phase which involves calculating some additional columns, applying calibrations, filtering out the 'interesting' events and extracting histograms describing those events. Conceptually, it's an iterative process of "extract plots, look at plots, change parameters", but running in a batch system means the latency is bad, so it can take a long time to converge to the right set of params.
>
> ** though we have much smaller, dedicated k8s/mesos/yarn clusters we use for prototyping
>
>>
>> Thanks,
>>
>> Xingbo
>>
>> On Fri, Mar 13, 2020 at 10:23 AM Sean Owen <sr...@gmail.com> wrote:
>>>
>>> You have multiple workers in one Spark (standalone) app? this wouldn't
>>> prevent N apps from each having a worker on a machine.
>>>
>>> On Fri, Mar 13, 2020 at 11:51 AM Andrew Melo <an...@gmail.com> wrote:
>>> >
>>> > Hello,
>>> >
>>> > On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com> wrote:
>>> >>
>>> >> Hi all,
>>> >>
>>> >> Based on my experience, there is no scenario that necessarily requires deploying multiple Workers on the same node with Standalone backend. A worker should book all the resources reserved to Spark on the host it is launched, then it can allocate those resources to one or more executors launched by this worker. Since each executor runs in a separated JVM, we can limit the memory of each executor to avoid long GC pause.
>>> >>
>>> >> The remaining concern is the local-cluster mode is implemented by launching multiple workers on the local host, we might need to re-implement LocalSparkCluster to launch only one Worker and multiple executors. It should be fine because local-cluster mode is only used in running Spark unit test cases, thus end users should not be affected by this change.
>>> >>
>>> >> Removing multiple workers on the same host support could simplify the deploy model of Standalone backend, and also reduce the burden to support legacy deploy pattern in the future feature developments. (There is an example in https://issues.apache.org/jira/browse/SPARK-27371 , where we designed a complex approach to coordinate resource requirements from different workers launched on the same host).
>>> >>
>>> >> The proposal is to update the document to deprecate the support of system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the support in the next major version (Spark 3.1).
>>> >>
>>> >> Please kindly let me know if you have use cases relying on this feature.
>>> >
>>> >
>>> > When deploying spark on batch systems (by wrapping the standalone deployment in scripts that can be consumed by the batch scheduler), we typically end up with >1 worker per host. If I understand correctly, this proposal would make our use case unsupported.
>>> >
>>> > Thanks,
>>> > Andrew
>>> >
>>> >
>>> >
>>> >>
>>> >> Thanks!
>>> >>
>>> >> Xingbo
>>> >
>>> > --
>>> > It's dark in this basement.
>>>
>>> ---------------------------------------------------------------------
>>> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org
>>>

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org


Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Andrew Melo <an...@gmail.com>.
Hi Xingbo, Sean,

On Fri, Mar 13, 2020 at 12:31 PM Xingbo Jiang <ji...@gmail.com> wrote:

> Andrew, could you provide more context of your use case please? Is it like
> you deploy homogeneous containers on hosts with available resources, and
> each container launches one worker? Or you deploy workers directly on hosts
> thus you could have multiple workers from the same application on the same
> host?
>

Sure, I describe a bit more detail about the actual workload below [*], but
the short version is that our computing resources/infrastructure are all
built around batch submission into (usually) the HTCondor scheduler, and
we've got a PoC using pyspark to replace the interactive portion of data
analysis. To run pyspark on our main resources, we use some scripts around
the standalone mode to spin up N slaves per-user**, which may or may not
end up on the same host. I understood Xingbo's original mail to mean that
wouldn't be allowed in the future, but from Sean's response, it seems like
I'm incorrect.

That being said, our use-case is very bursty, and it would be very good if
there was a way we could have one pool of JVMs that could be shared between
N different concurrent users instead of having N different pools of JVMs
that each serve one person. We're already resource constrained, and we're
expecting our data rates to increase 10x in 2026, so the less idle CPU, the
better for us.

Andrew

* I work for one of the LHC experiments at CERN (https://cms.cern/)
and there's two main "phases" of our data pipeline: production and
analysis. The analysis half is currently implemented by having users
writing some software, splitting the input dataset(s) into N parts and then
submitting those jobs to the batch system (combining the outputs in a
manual postprocessing step). In terms of scale, there are currently ~100
users running ~900 tasks over ~50k cpus. The use case relevant to this
context is the terminal analysis phase which involves calculating some
additional columns, applying calibrations, filtering out the 'interesting'
events and extracting histograms describing those events. Conceptually,
it's an iterative process of "extract plots, look at plots, change
parameters", but running in a batch system means the latency is bad, so it
can take a long time to converge to the right set of params.

** though we have much smaller, dedicated k8s/mesos/yarn clusters we use
for prototyping


> Thanks,
>
> Xingbo
>
> On Fri, Mar 13, 2020 at 10:23 AM Sean Owen <sr...@gmail.com> wrote:
>
>> You have multiple workers in one Spark (standalone) app? this wouldn't
>> prevent N apps from each having a worker on a machine.
>>
>> On Fri, Mar 13, 2020 at 11:51 AM Andrew Melo <an...@gmail.com>
>> wrote:
>> >
>> > Hello,
>> >
>> > On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com>
>> wrote:
>> >>
>> >> Hi all,
>> >>
>> >> Based on my experience, there is no scenario that necessarily requires
>> deploying multiple Workers on the same node with Standalone backend. A
>> worker should book all the resources reserved to Spark on the host it is
>> launched, then it can allocate those resources to one or more executors
>> launched by this worker. Since each executor runs in a separated JVM, we
>> can limit the memory of each executor to avoid long GC pause.
>> >>
>> >> The remaining concern is the local-cluster mode is implemented by
>> launching multiple workers on the local host, we might need to re-implement
>> LocalSparkCluster to launch only one Worker and multiple executors. It
>> should be fine because local-cluster mode is only used in running Spark
>> unit test cases, thus end users should not be affected by this change.
>> >>
>> >> Removing multiple workers on the same host support could simplify the
>> deploy model of Standalone backend, and also reduce the burden to support
>> legacy deploy pattern in the future feature developments. (There is an
>> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
>> designed a complex approach to coordinate resource requirements from
>> different workers launched on the same host).
>> >>
>> >> The proposal is to update the document to deprecate the support of
>> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
>> support in the next major version (Spark 3.1).
>> >>
>> >> Please kindly let me know if you have use cases relying on this
>> feature.
>> >
>> >
>> > When deploying spark on batch systems (by wrapping the standalone
>> deployment in scripts that can be consumed by the batch scheduler), we
>> typically end up with >1 worker per host. If I understand correctly, this
>> proposal would make our use case unsupported.
>> >
>> > Thanks,
>> > Andrew
>> >
>> >
>> >
>> >>
>> >> Thanks!
>> >>
>> >> Xingbo
>> >
>> > --
>> > It's dark in this basement.
>>
>> ---------------------------------------------------------------------
>> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org
>>
>>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Xingbo Jiang <ji...@gmail.com>.
Andrew, could you provide more context of your use case please? Is it like
you deploy homogeneous containers on hosts with available resources, and
each container launches one worker? Or you deploy workers directly on hosts
thus you could have multiple workers from the same application on the same
host?

Thanks,

Xingbo

On Fri, Mar 13, 2020 at 10:23 AM Sean Owen <sr...@gmail.com> wrote:

> You have multiple workers in one Spark (standalone) app? this wouldn't
> prevent N apps from each having a worker on a machine.
>
> On Fri, Mar 13, 2020 at 11:51 AM Andrew Melo <an...@gmail.com>
> wrote:
> >
> > Hello,
> >
> > On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com>
> wrote:
> >>
> >> Hi all,
> >>
> >> Based on my experience, there is no scenario that necessarily requires
> deploying multiple Workers on the same node with Standalone backend. A
> worker should book all the resources reserved to Spark on the host it is
> launched, then it can allocate those resources to one or more executors
> launched by this worker. Since each executor runs in a separated JVM, we
> can limit the memory of each executor to avoid long GC pause.
> >>
> >> The remaining concern is the local-cluster mode is implemented by
> launching multiple workers on the local host, we might need to re-implement
> LocalSparkCluster to launch only one Worker and multiple executors. It
> should be fine because local-cluster mode is only used in running Spark
> unit test cases, thus end users should not be affected by this change.
> >>
> >> Removing multiple workers on the same host support could simplify the
> deploy model of Standalone backend, and also reduce the burden to support
> legacy deploy pattern in the future feature developments. (There is an
> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
> designed a complex approach to coordinate resource requirements from
> different workers launched on the same host).
> >>
> >> The proposal is to update the document to deprecate the support of
> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
> support in the next major version (Spark 3.1).
> >>
> >> Please kindly let me know if you have use cases relying on this feature.
> >
> >
> > When deploying spark on batch systems (by wrapping the standalone
> deployment in scripts that can be consumed by the batch scheduler), we
> typically end up with >1 worker per host. If I understand correctly, this
> proposal would make our use case unsupported.
> >
> > Thanks,
> > Andrew
> >
> >
> >
> >>
> >> Thanks!
> >>
> >> Xingbo
> >
> > --
> > It's dark in this basement.
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org
>
>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Sean Owen <sr...@gmail.com>.
You have multiple workers in one Spark (standalone) app? this wouldn't
prevent N apps from each having a worker on a machine.

On Fri, Mar 13, 2020 at 11:51 AM Andrew Melo <an...@gmail.com> wrote:
>
> Hello,
>
> On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com> wrote:
>>
>> Hi all,
>>
>> Based on my experience, there is no scenario that necessarily requires deploying multiple Workers on the same node with Standalone backend. A worker should book all the resources reserved to Spark on the host it is launched, then it can allocate those resources to one or more executors launched by this worker. Since each executor runs in a separated JVM, we can limit the memory of each executor to avoid long GC pause.
>>
>> The remaining concern is the local-cluster mode is implemented by launching multiple workers on the local host, we might need to re-implement LocalSparkCluster to launch only one Worker and multiple executors. It should be fine because local-cluster mode is only used in running Spark unit test cases, thus end users should not be affected by this change.
>>
>> Removing multiple workers on the same host support could simplify the deploy model of Standalone backend, and also reduce the burden to support legacy deploy pattern in the future feature developments. (There is an example in https://issues.apache.org/jira/browse/SPARK-27371 , where we designed a complex approach to coordinate resource requirements from different workers launched on the same host).
>>
>> The proposal is to update the document to deprecate the support of system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the support in the next major version (Spark 3.1).
>>
>> Please kindly let me know if you have use cases relying on this feature.
>
>
> When deploying spark on batch systems (by wrapping the standalone deployment in scripts that can be consumed by the batch scheduler), we typically end up with >1 worker per host. If I understand correctly, this proposal would make our use case unsupported.
>
> Thanks,
> Andrew
>
>
>
>>
>> Thanks!
>>
>> Xingbo
>
> --
> It's dark in this basement.

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org


Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Andrew Melo <an...@gmail.com>.
Hello,

On Fri, Feb 28, 2020 at 13:21 Xingbo Jiang <ji...@gmail.com> wrote:

> Hi all,
>
> Based on my experience, there is no scenario that necessarily requires
> deploying multiple Workers on the same node with Standalone backend. A
> worker should book all the resources reserved to Spark on the host it is
> launched, then it can allocate those resources to one or more executors
> launched by this worker. Since each executor runs in a separated JVM, we
> can limit the memory of each executor to avoid long GC pause.
>
> The remaining concern is the local-cluster mode is implemented by
> launching multiple workers on the local host, we might need to re-implement
> LocalSparkCluster to launch only one Worker and multiple executors. It
> should be fine because local-cluster mode is only used in running Spark
> unit test cases, thus end users should not be affected by this change.
>
> Removing multiple workers on the same host support could simplify the
> deploy model of Standalone backend, and also reduce the burden to support
> legacy deploy pattern in the future feature developments. (There is an
> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
> designed a complex approach to coordinate resource requirements from
> different workers launched on the same host).
>
> The proposal is to update the document to deprecate the support of system
> environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the support
> in the next major version (Spark 3.1).
>
> Please kindly let me know if you have use cases relying on this feature.
>

When deploying spark on batch systems (by wrapping the standalone
deployment in scripts that can be consumed by the batch scheduler), we
typically end up with >1 worker per host. If I understand correctly, this
proposal would make our use case unsupported.

Thanks,
Andrew




> Thanks!
>
> Xingbo
>
-- 
It's dark in this basement.

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Xingbo Jiang <ji...@gmail.com>.
Hi Prashant,

I guess you are referring to the local-cluster mode? AFAIK the
local-cluster mode has not been mentioned at all in the user guide, thus it
should only be used in Spark tests. Also, there are a few differences
between having multiple workers on the same node and having one worker on
each node, as I mentioned in
https://issues.apache.org/jira/browse/SPARK-27371 , a complex approach is
needed to resolve the resource requirement contentions between different
workers running on the same node.

Cheers,

Xingbo

On Thu, Mar 5, 2020 at 8:49 PM Prashant Sharma <sc...@gmail.com> wrote:

> It was by design, one could run multiple workers on his laptop for trying
> out or testing spark in distributed mode, one could launch multiple workers
> and see how resource offers and requirements work. Certainly, I have not
> commonly seen, starting multiple workers on the same node as a practice so
> far.
>
> Why do we consider it as a special case for scheduling, where two workers
> are on the same node than two different nodes? Possibly, optimize on
> network I/o and disk I/O?
>
> On Tue, Mar 3, 2020 at 12:45 AM Xingbo Jiang <ji...@gmail.com>
> wrote:
>
>> Thanks Sean for your input, I really think it could simplify Spark
>> Standalone backend a lot by only allowing a single worker on the same host,
>> also I can confirm this deploy model can satisfy all the workloads deployed
>> on Standalone backend AFAIK.
>>
>> Regarding the case multiple distinct Spark clusters running a worker on
>> one machine, I'm not sure whether that's something we have claimed to
>> support, could someone with more context on this scenario share their use
>> case?
>>
>> Cheers,
>>
>> Xingbo
>>
>> On Fri, Feb 28, 2020 at 11:29 AM Sean Owen <sr...@gmail.com> wrote:
>>
>>> I'll admit, I didn't know you could deploy multiple workers per
>>> machine. I agree, I don't see the use case for it? multiple executors,
>>> yes of course. And I guess you could imagine multiple distinct Spark
>>> clusters running a worker on one machine. I don't have an informed
>>> opinion therefore, but agree that it seems like a best practice enough
>>> to enforce 1 worker per machine, if it makes things simpler rather
>>> than harder.
>>>
>>> On Fri, Feb 28, 2020 at 1:21 PM Xingbo Jiang <ji...@gmail.com>
>>> wrote:
>>> >
>>> > Hi all,
>>> >
>>> > Based on my experience, there is no scenario that necessarily requires
>>> deploying multiple Workers on the same node with Standalone backend. A
>>> worker should book all the resources reserved to Spark on the host it is
>>> launched, then it can allocate those resources to one or more executors
>>> launched by this worker. Since each executor runs in a separated JVM, we
>>> can limit the memory of each executor to avoid long GC pause.
>>> >
>>> > The remaining concern is the local-cluster mode is implemented by
>>> launching multiple workers on the local host, we might need to re-implement
>>> LocalSparkCluster to launch only one Worker and multiple executors. It
>>> should be fine because local-cluster mode is only used in running Spark
>>> unit test cases, thus end users should not be affected by this change.
>>> >
>>> > Removing multiple workers on the same host support could simplify the
>>> deploy model of Standalone backend, and also reduce the burden to support
>>> legacy deploy pattern in the future feature developments. (There is an
>>> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
>>> designed a complex approach to coordinate resource requirements from
>>> different workers launched on the same host).
>>> >
>>> > The proposal is to update the document to deprecate the support of
>>> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
>>> support in the next major version (Spark 3.1).
>>> >
>>> > Please kindly let me know if you have use cases relying on this
>>> feature.
>>> >
>>> > Thanks!
>>> >
>>> > Xingbo
>>>
>>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Prashant Sharma <sc...@gmail.com>.
It was by design, one could run multiple workers on his laptop for trying
out or testing spark in distributed mode, one could launch multiple workers
and see how resource offers and requirements work. Certainly, I have not
commonly seen, starting multiple workers on the same node as a practice so
far.

Why do we consider it as a special case for scheduling, where two workers
are on the same node than two different nodes? Possibly, optimize on
network I/o and disk I/O?

On Tue, Mar 3, 2020 at 12:45 AM Xingbo Jiang <ji...@gmail.com> wrote:

> Thanks Sean for your input, I really think it could simplify Spark
> Standalone backend a lot by only allowing a single worker on the same host,
> also I can confirm this deploy model can satisfy all the workloads deployed
> on Standalone backend AFAIK.
>
> Regarding the case multiple distinct Spark clusters running a worker on
> one machine, I'm not sure whether that's something we have claimed to
> support, could someone with more context on this scenario share their use
> case?
>
> Cheers,
>
> Xingbo
>
> On Fri, Feb 28, 2020 at 11:29 AM Sean Owen <sr...@gmail.com> wrote:
>
>> I'll admit, I didn't know you could deploy multiple workers per
>> machine. I agree, I don't see the use case for it? multiple executors,
>> yes of course. And I guess you could imagine multiple distinct Spark
>> clusters running a worker on one machine. I don't have an informed
>> opinion therefore, but agree that it seems like a best practice enough
>> to enforce 1 worker per machine, if it makes things simpler rather
>> than harder.
>>
>> On Fri, Feb 28, 2020 at 1:21 PM Xingbo Jiang <ji...@gmail.com>
>> wrote:
>> >
>> > Hi all,
>> >
>> > Based on my experience, there is no scenario that necessarily requires
>> deploying multiple Workers on the same node with Standalone backend. A
>> worker should book all the resources reserved to Spark on the host it is
>> launched, then it can allocate those resources to one or more executors
>> launched by this worker. Since each executor runs in a separated JVM, we
>> can limit the memory of each executor to avoid long GC pause.
>> >
>> > The remaining concern is the local-cluster mode is implemented by
>> launching multiple workers on the local host, we might need to re-implement
>> LocalSparkCluster to launch only one Worker and multiple executors. It
>> should be fine because local-cluster mode is only used in running Spark
>> unit test cases, thus end users should not be affected by this change.
>> >
>> > Removing multiple workers on the same host support could simplify the
>> deploy model of Standalone backend, and also reduce the burden to support
>> legacy deploy pattern in the future feature developments. (There is an
>> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
>> designed a complex approach to coordinate resource requirements from
>> different workers launched on the same host).
>> >
>> > The proposal is to update the document to deprecate the support of
>> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
>> support in the next major version (Spark 3.1).
>> >
>> > Please kindly let me know if you have use cases relying on this feature.
>> >
>> > Thanks!
>> >
>> > Xingbo
>>
>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Xingbo Jiang <ji...@gmail.com>.
Thanks Sean for your input, I really think it could simplify Spark
Standalone backend a lot by only allowing a single worker on the same host,
also I can confirm this deploy model can satisfy all the workloads deployed
on Standalone backend AFAIK.

Regarding the case multiple distinct Spark clusters running a worker on one
machine, I'm not sure whether that's something we have claimed to support,
could someone with more context on this scenario share their use case?

Cheers,

Xingbo

On Fri, Feb 28, 2020 at 11:29 AM Sean Owen <sr...@gmail.com> wrote:

> I'll admit, I didn't know you could deploy multiple workers per
> machine. I agree, I don't see the use case for it? multiple executors,
> yes of course. And I guess you could imagine multiple distinct Spark
> clusters running a worker on one machine. I don't have an informed
> opinion therefore, but agree that it seems like a best practice enough
> to enforce 1 worker per machine, if it makes things simpler rather
> than harder.
>
> On Fri, Feb 28, 2020 at 1:21 PM Xingbo Jiang <ji...@gmail.com>
> wrote:
> >
> > Hi all,
> >
> > Based on my experience, there is no scenario that necessarily requires
> deploying multiple Workers on the same node with Standalone backend. A
> worker should book all the resources reserved to Spark on the host it is
> launched, then it can allocate those resources to one or more executors
> launched by this worker. Since each executor runs in a separated JVM, we
> can limit the memory of each executor to avoid long GC pause.
> >
> > The remaining concern is the local-cluster mode is implemented by
> launching multiple workers on the local host, we might need to re-implement
> LocalSparkCluster to launch only one Worker and multiple executors. It
> should be fine because local-cluster mode is only used in running Spark
> unit test cases, thus end users should not be affected by this change.
> >
> > Removing multiple workers on the same host support could simplify the
> deploy model of Standalone backend, and also reduce the burden to support
> legacy deploy pattern in the future feature developments. (There is an
> example in https://issues.apache.org/jira/browse/SPARK-27371 , where we
> designed a complex approach to coordinate resource requirements from
> different workers launched on the same host).
> >
> > The proposal is to update the document to deprecate the support of
> system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the
> support in the next major version (Spark 3.1).
> >
> > Please kindly let me know if you have use cases relying on this feature.
> >
> > Thanks!
> >
> > Xingbo
>

Re: [DISCUSS] Remove multiple workers on the same host support from Standalone backend

Posted by Sean Owen <sr...@gmail.com>.
I'll admit, I didn't know you could deploy multiple workers per
machine. I agree, I don't see the use case for it? multiple executors,
yes of course. And I guess you could imagine multiple distinct Spark
clusters running a worker on one machine. I don't have an informed
opinion therefore, but agree that it seems like a best practice enough
to enforce 1 worker per machine, if it makes things simpler rather
than harder.

On Fri, Feb 28, 2020 at 1:21 PM Xingbo Jiang <ji...@gmail.com> wrote:
>
> Hi all,
>
> Based on my experience, there is no scenario that necessarily requires deploying multiple Workers on the same node with Standalone backend. A worker should book all the resources reserved to Spark on the host it is launched, then it can allocate those resources to one or more executors launched by this worker. Since each executor runs in a separated JVM, we can limit the memory of each executor to avoid long GC pause.
>
> The remaining concern is the local-cluster mode is implemented by launching multiple workers on the local host, we might need to re-implement LocalSparkCluster to launch only one Worker and multiple executors. It should be fine because local-cluster mode is only used in running Spark unit test cases, thus end users should not be affected by this change.
>
> Removing multiple workers on the same host support could simplify the deploy model of Standalone backend, and also reduce the burden to support legacy deploy pattern in the future feature developments. (There is an example in https://issues.apache.org/jira/browse/SPARK-27371 , where we designed a complex approach to coordinate resource requirements from different workers launched on the same host).
>
> The proposal is to update the document to deprecate the support of system environment `SPARK_WORKER_INSTANCES` in Spark 3.0, and remove the support in the next major version (Spark 3.1).
>
> Please kindly let me know if you have use cases relying on this feature.
>
> Thanks!
>
> Xingbo

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org