Re: Clarification on ExecutorRoll Plugin & Ignore Decommission Fetch Failure

2023-08-25 Thread Dongjoon Hyun
Hi, Arun.

Here are some answers to your questions.

First, the fetch failure is irrelevant to the Executor Rolling feature
because the plugin itself only asked the Spark scheduler to decommission
it, not terminate it. More specifically, it's independent from the
underlying Decommissioning feature's behavior. FYI, the following is the
code. In other words, it's totally a behavior of the storage
decommissioning feature and `spark.stage.ignoreDecommissionFetchFailure`
configuration.

https://github.com/apache/spark/blob/12f3c81c26ef639842b8a155e5fd5ccfa7705bea/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorRollPlugin.scala#L84

Second, for the following your comment,
`spark.stage.ignoreDecommissionFetchFailure` is not designed to prevent
FetchFailure. As you see in the documentation, it tries to ignore stage
fetch failure caused by executor decommission during counting
spark.stage.maxConsecutiveAttempts. Here is SPARK-40481 PR for details.

> I notice that there are shuffle fetch failures in tasks and the above
ignore decommission
> configurations are not respected. The stage will go into retry. The
decommissioned
> executor logs clearly show the decommission was fully graceful and blocks
were replicated
> to other active executors/fallback.

https://github.com/apache/spark/pull/37924
[SPARK-40481][CORE] Ignore stage fetch failure caused by decommissioned
executor

Lastly, SPARK-40481 was not designed as a silver bullet from the beginning.
Instead, it was a best effort approach at that time. The limitation was
pointed out during the PR review and the PR description has the following
warning.

> Fetch failure might not be ignored when executors are in below condition,
> but this is best effort approach based on current mechanism.
> Stopped or terminated after finishing decommission
> Under decommission process, then removed by driver with other reasons


Dongjoon.



On Fri, Aug 25, 2023 at 8:21 AM Arun Ravi  wrote:

> Hi Team,
> I am running Apache Spark  3.4.1 Application on K8s with the below
> configuration related to executor rolling and Ignore Decommission Fetch
> Failure.
>
> spark.plugins: "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin"
> spark.kubernetes.executor.rollInterval: "1800s"
> spark.kubernetes.executor.rollPolicy: "OUTLIER_NO_FALLBACK"
> spark.kubernetes.executor.minTasksPerExecutorBeforeRolling: "100"
>
> spark.stage.ignoreDecommissionFetchFailure: "true"
> spark.scheduler.maxRetainedRemovedDecommissionExecutors: "20"
>
> spark.decommission.enabled: "true"
> spark.storage.decommission.enabled: "true"
> spark.storage.decommission.fallbackStorage.path: "some-s3-path"
> spark.storage.decommission.shuffleBlocks.maxThreads: "16"
>
> When an executor is decommissioned in the middle of the stage, I notice
> that there are shuffle fetch failures in tasks and the above ignore
> decommission configurations are not respected. The stage will go into
> retry. The decommissioned executor logs clearly show the decommission was
> fully graceful and blocks were replicated to other active
> executors/fallback.
>
> May I know how I should be using Executor Rolling, without triggering
> stage failures? I am using executor rolling to avoid executors being
> removed by K8s due to memory pressure or oom issues as my spark job is
> heavy on shuffling and has a lot of window functions. Any help will be
> super useful.
>
>
>
> Arun Ravi M V
> B.Tech (Batch: 2010-2014)
>
> Computer Science and Engineering
>
> Govt. Model Engineering College
> Cochin University Of Science And Technology
> Kochi
>


Re: Clarification on ExecutorRoll Plugin & Ignore Decommission Fetch Failure

2023-08-25 Thread Mich Talebzadeh
Hi,

The crux of the matter here as I understand is " how should I be using
Executor Rolling, without triggering stage failures?"

The object of executor rolling is to replace decommissioning executors with
new ones while minimizing the impact on running tasks and stages. in k8s.

As mentioned

spark.plugins: "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin"
spark.kubernetes.executor.rollInterval: "1800s"
spark.kubernetes.executor.rollPolicy: "OUTLIER_NO_FALLBACK"
spark.kubernetes.executor.minTasksPerExecutorBeforeRolling: "100"

You will need to ensure that  the decommissioning of executors is done
gracefully. |As per classic Spark, data and tasks being handled by a
decommissioned executor should be properly redistributed to active
executors before the decommissioned executor is removed, otherwise you are
going to have issues. Need to have an eye on fetch failures during
rolling. *This
can happen if tasks attempt to fetch data from decommissioned executors
before the data is redistributed.*
Possible remedy would be to set
"spark.stage.ignoreDecommissionFetchFailure'', "true" (as you have
correctly pointed out)  to tell Spark to ignore fetch failures from
decommissioned executors and retry the tasks on the remaining active
executors as per norm. This will incur additional computation as expected
but will ensure data integrity

In general other parameters settings such as
"spark.kubernetes.executor.minTasksPerExecutorBeforeRolling" need to be
tried for your workload and it is practically impossible to guess for
optimum values. This parameter controls the minimum number of tasks that
should be completed before an executor is rolled.

HTH

Mich Talebzadeh,
Distinguished Technologist, Solutions Architect & Engineer
London
United Kingdom


   view my Linkedin profile



 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Fri, 25 Aug 2023 at 17:48, Arun Ravi  wrote:

> Hi Team,
> I am running Apache Spark  3.4.1 Application on K8s with the below
> configuration related to executor rolling and Ignore Decommission Fetch
> Failure.
>
> spark.plugins: "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin"
> spark.kubernetes.executor.rollInterval: "1800s"
> spark.kubernetes.executor.rollPolicy: "OUTLIER_NO_FALLBACK"
> spark.kubernetes.executor.minTasksPerExecutorBeforeRolling: "100"
>
> spark.stage.ignoreDecommissionFetchFailure: "true"
> spark.scheduler.maxRetainedRemovedDecommissionExecutors: "20"
>
> spark.decommission.enabled: "true"
> spark.storage.decommission.enabled: "true"
> spark.storage.decommission.fallbackStorage.path: "some-s3-path"
> spark.storage.decommission.shuffleBlocks.maxThreads: "16"
>
> When an executor is decommissioned in the middle of the stage, I notice
> that there are shuffle fetch failures in tasks and the above ignore
> decommission configurations are not respected. The stage will go into
> retry. The decommissioned executor logs clearly show the decommission was
> fully graceful and blocks were replicated to other active
> executors/fallback.
>
> May I know how I should be using Executor Rolling, without triggering
> stage failures? I am using executor rolling to avoid executors being
> removed by K8s due to memory pressure or oom issues as my spark job is
> heavy on shuffling and has a lot of window functions. Any help will be
> super useful.
>
>
>
> Arun Ravi M V
> B.Tech (Batch: 2010-2014)
>
> Computer Science and Engineering
>
> Govt. Model Engineering College
> Cochin University Of Science And Technology
> Kochi
>


unsubscribe

2023-08-25 Thread Nizam Shaik
unsubscribe


Re: [Internet]Re: Improving Dynamic Allocation Logic for Spark 4+

2023-08-25 Thread Mich Talebzadeh
Hi Qian,

How in practice have you implemented image caching for the driver and
executor pods respectively?

Thanks

On Thu, 24 Aug 2023 at 02:44, Qian Sun  wrote:

> Hi Mich
>
> I agree with your opinion that the startup time of the Spark on Kubernetes
> cluster needs to be improved.
>
> Regarding the fetching image directly, I have utilized ImageCache to store
> the images on the node, eliminating the time required to pull images from a
> remote repository, which does indeed lead to a reduction in overall time,
> and the effect becomes more pronounced as the size of the image increases.
>
>
> Additionally, I have observed that the driver pod takes a significant
> amount of time from running to attempting to create executor pods, with an
> estimated time expenditure of around 75%. We can also explore optimization
> options in this area.
>
> On Thu, Aug 24, 2023 at 12:58 AM Mich Talebzadeh <
> mich.talebza...@gmail.com> wrote:
>
>> Hi all,
>>
>> On this conversion, one of the issues I brought up was the driver
>> start-up time. This is especially true in k8s. As spark on k8s is modeled
>> on Spark on standalone schedler, Spark on k8s consist of a single-driver
>> pod (as master on standalone”) and a  number of executors (“workers”). When 
>> executed
>> on k8s, the driver and executors are executed on separate pods
>> . First
>> the driver pod is launched, then the driver pod itself launches the
>> executor pods. From my observation, in an auto scaling cluster, the driver
>> pod may take up to 40 seconds followed by executor pods. This is a
>> considerable time for customers and it is painfully slow. Can we actually
>> move away from dependency on standalone mode and try to speed up k8s
>> cluster formation.
>>
>> Another naive question, when the docker image is pulled from the
>> container registry to the driver itself, this takes finite time. The docker
>> image for executors could be different from that of the driver
>> docker image. Since spark-submit presents this at the time of submission,
>> can we save time by fetching the docker images straight away?
>>
>> Thanks
>>
>> Mich
>>
>>
>>view my Linkedin profile
>> 
>>
>>
>>  https://en.everybodywiki.com/Mich_Talebzadeh
>>
>>
>>
>> *Disclaimer:* Use it at your own risk. Any and all responsibility for
>> any loss, damage or destruction of data or any other property which may
>> arise from relying on this email's technical content is explicitly
>> disclaimed. The author will in no case be liable for any monetary damages
>> arising from such loss, damage or destruction.
>>
>>
>>
>>
>> On Tue, 8 Aug 2023 at 18:25, Mich Talebzadeh 
>> wrote:
>>
>>> Splendid idea. 
>>>
>>> Mich Talebzadeh,
>>> Solutions Architect/Engineering Lead
>>> London
>>> United Kingdom
>>>
>>>
>>>view my Linkedin profile
>>> 
>>>
>>>
>>>  https://en.everybodywiki.com/Mich_Talebzadeh
>>>
>>>
>>>
>>> *Disclaimer:* Use it at your own risk. Any and all responsibility for
>>> any loss, damage or destruction of data or any other property which may
>>> arise from relying on this email's technical content is explicitly
>>> disclaimed. The author will in no case be liable for any monetary damages
>>> arising from such loss, damage or destruction.
>>>
>>>
>>>
>>>
>>> On Tue, 8 Aug 2023 at 18:10, Holden Karau  wrote:
>>>
 The driver it’s self is probably another topic, perhaps I’ll make a
 “faster spark star time” JIRA and a DA JIRA and we can explore both.

 On Tue, Aug 8, 2023 at 10:07 AM Mich Talebzadeh <
 mich.talebza...@gmail.com> wrote:

> From my own perspective faster execution time especially with Spark on
> tin boxes (Dataproc & EC2) and Spark on k8s is something that customers
> often bring up.
>
> Poor time to onboard with autoscaling seems to be particularly singled
> out for heavy ETL jobs that use Spark. I am disappointed to see the poor
> performance of Spark on k8s autopilot with timelines starting the driver
> itself and moving from Pending to Running phase (Spark 4.3.1 with Java 11)
>
> HTH
>
> Mich Talebzadeh,
> Solutions Architect/Engineering Lead
> London
> United Kingdom
>
>
>view my Linkedin profile
> 
>
>
>  https://en.everybodywiki.com/Mich_Talebzadeh
>
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for
> any loss, damage or destruction of data or any other property which may
> arise from relying on this email's technical content is explicitly
> disclaimed. The author will in no case be liable for any monetary damages
> arising from such loss, damage or destruction.
>
>
>
>
> On Tue, 8 Aug 2023 at 15:49, kalyan  wrote:
>
>> +1 to 

Clarification on ExecutorRoll Plugin & Ignore Decommission Fetch Failure

2023-08-25 Thread Arun Ravi
Hi Team,
I am running Apache Spark  3.4.1 Application on K8s with the below
configuration related to executor rolling and Ignore Decommission Fetch
Failure.

spark.plugins: "org.apache.spark.scheduler.cluster.k8s.ExecutorRollPlugin"
spark.kubernetes.executor.rollInterval: "1800s"
spark.kubernetes.executor.rollPolicy: "OUTLIER_NO_FALLBACK"
spark.kubernetes.executor.minTasksPerExecutorBeforeRolling: "100"

spark.stage.ignoreDecommissionFetchFailure: "true"
spark.scheduler.maxRetainedRemovedDecommissionExecutors: "20"

spark.decommission.enabled: "true"
spark.storage.decommission.enabled: "true"
spark.storage.decommission.fallbackStorage.path: "some-s3-path"
spark.storage.decommission.shuffleBlocks.maxThreads: "16"

When an executor is decommissioned in the middle of the stage, I notice
that there are shuffle fetch failures in tasks and the above ignore
decommission configurations are not respected. The stage will go into
retry. The decommissioned executor logs clearly show the decommission was
fully graceful and blocks were replicated to other active
executors/fallback.

May I know how I should be using Executor Rolling, without triggering stage
failures? I am using executor rolling to avoid executors being removed by
K8s due to memory pressure or oom issues as my spark job is heavy on
shuffling and has a lot of window functions. Any help will be super useful.



Arun Ravi M V
B.Tech (Batch: 2010-2014)

Computer Science and Engineering

Govt. Model Engineering College
Cochin University Of Science And Technology
Kochi


Apache Spark 4.0.0-SNAPSHOT is ready for Java 21

2023-08-25 Thread Dongjoon Hyun
Hi, All.

Java 21 will be released in a month and Apache Spark master branch
(4.0.0-SNAPSHOT) achieved the first milestone (SPARK-43831: Build and Run
Spark on Java 21) Today.

1. JDK 21: https://openjdk.org/projects/jdk/21/

  - 2023/08/24 Final Release Candidate
  - 2023/09/19 General Availability

2. `master` branch CIs

We have a healthy SBT-based Daily Java 21 Test Coverage
(Java/Scala/R/Python). Also, the main CI (PR and commit) job has Java 21
Maven Build Test Coverage in addition to Java 11 and 17 ones.

  - Daily CI:
https://github.com/apache/spark/actions/workflows/build_java21.yml
  - Main CI:
https://github.com/apache/spark/actions/workflows/build_main.yml?query=branch%3Amaster

You can play with Spark on Java 21. Please let us know if you have any
issues.

Thanks,
Dongjoon.