Do you know by any chance if that config also applies to Databricks?

El jue, 10 oct 2024 a las 10:02, Ángel (<angel.alvarez.pas...@gmail.com>)
escribió:

> Thanks a lot for the clarification. Interesting... I've never needed it,
> even though I've been using Spark for over 8 years.
>
> El jue, 10 oct 2024 a las 9:21, Liu Cao (<twcnnj...@gmail.com>) escribió:
>
>> I’m unclear on what the exact issue the OP ran into.
>>
>> But if we are talking about decommission, just one side note:
>>
>> The decommission feature
>> <https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE/edit?tab=t.0#heading=h.70pylrl2vdg8>
>> has been in spark for a while, and decommission on K8S and standalone was
>> GA-ed in 3.1 actually. See
>> https://spark.apache.org/releases/spark-release-3-1-1.html
>>
>> The fact that you didn’t see it in the 3.3 site is simply a lack of
>> documentation. The missing documentation was added in 3.4, thanks to
>> https://github.com/apache/spark/pull/38131/files
>>
>> On Wed, Oct 9, 2024 at 10:13 PM Ángel <angel.alvarez.pas...@gmail.com>
>> wrote:
>>
>>> The Synapse config "spark.yarn.executor.decommission.enabled"  is the
>>> closest thing to the proposed config "spark.executor.decommission.enabled"
>>> we've seen so far, I was only remarking that.
>>>
>>> On the other hand, seems like the "decomission" config came out in Spark
>>> 3.4.0:
>>>
>>> https://archive.apache.org/dist/spark/docs/3.3.4/configuration.html
>>> https://archive.apache.org/dist/spark/docs/3.4.0/configuration.html
>>>
>>>
>>>
>>> El jue, 10 oct 2024 a las 4:44, Jungtaek Lim (<
>>> kabhwan.opensou...@gmail.com>) escribió:
>>>
>>>> Ángel,
>>>>
>>>> https://spark.apache.org/docs/latest/configuration.html
>>>> search through `decommission` in this page.
>>>>
>>>> The config you may have found from Synapse is
>>>> spark."yarn".executor.decommission.enabled. And the question was even "k8s"
>>>> and none of the information for the vendor was mentioned from the question.
>>>> I don't even think these configs are on the internet - simply saying,
>>>> google them (be sure to wrap with double quotes).
>>>>
>>>> Interestingly, I wouldn't even expect the config for graceful shutdown
>>>> for decommission. The functionality Spark provides for "decommission" is
>>>> basically a "graceful shutdown" of the executor. It sounds to me as
>>>> redundant.
>>>>
>>>> On Thu, Oct 10, 2024 at 11:11 AM Ángel <angel.alvarez.pas...@gmail.com>
>>>> wrote:
>>>>
>>>>> Looks like it actually exists ... but only for the Spark Synapse
>>>>> implementation ...
>>>>>
>>>>>
>>>>> https://learn.microsoft.com/en-us/answers/questions/1496283/purpose-of-spark-yarn-executor-decommission-enable
>>>>>
>>>>>
>>>>> Jay Han was asking for some config on k8s, so .... we shouldn't bring
>>>>> this config to the table, should we?
>>>>>
>>>>> El jue, 10 oct 2024 a las 2:55, Sean Owen (<sro...@gmail.com>)
>>>>> escribió:
>>>>>
>>>>>> Mich: you can set any key-value pair you want in Spark config. It
>>>>>> doesn't mean it is a real flag that code reads.
>>>>>>
>>>>>> spark.conf.set("ham", "sandwich")
>>>>>> print(spark.conf.get("ham"))
>>>>>>
>>>>>> prints "sandwich"
>>>>>>
>>>>>> forceKillTimeout is a real config:
>>>>>>
>>>>>> https://github.com/apache/spark/blob/fed9a8da3d4187794161e0be325aa96be8487783/core/src/main/scala/org/apache/spark/internal/config/package.scala#L2394
>>>>>>
>>>>>> The others I cannot find, as in:
>>>>>>
>>>>>> https://github.com/search?q=repo%3Aapache%2Fspark%20spark.executor.decommission.gracefulShutdown&type=code
>>>>>>
>>>>>> If you're continuing to suggest these are real configs, where are you
>>>>>> finding those two in any docs or source?
>>>>>> Or what config were you thinking of if it's a typo?
>>>>>>
>>>>>> On Wed, Oct 9, 2024 at 5:14 PM Mich Talebzadeh <
>>>>>> mich.talebza...@gmail.com> wrote:
>>>>>>
>>>>>>> Let us take this for a ride using these so called non-existent
>>>>>>> configuration settings
>>>>>>>
>>>>>>> spark.executor.decommission.enabled=true
>>>>>>> spark.executor.decommission.gracefulShutdown=true
>>>>>>>
>>>>>>> Tested on Spark 3.4
>>>>>>>
>>>>>>> from pyspark.sql import SparkSession
>>>>>>> # Initialize a Spark session
>>>>>>> spark = SparkSession.builder \
>>>>>>>     .appName("Verifying Spark Configurations") \
>>>>>>>     .config("spark.executor.decommission.enabled", "true") \
>>>>>>>     .config("spark.executor.decommission.forceKillTimeout", "100s") \
>>>>>>>     .getOrCreate()
>>>>>>>
>>>>>>> # Access Spark context
>>>>>>> sc = spark.sparkContext
>>>>>>> # Set the log level to ERROR to reduce verbosity
>>>>>>> sc.setLogLevel("ERROR")
>>>>>>> print(f"\n\nSpark version: ", sc.version)
>>>>>>>
>>>>>>> # Verify the configuration for executor decommissioning
>>>>>>> decommission_enabled =
>>>>>>> sc.getConf().get("spark.executor.decommission.enabled", "false")
>>>>>>> force_kill_timeout =
>>>>>>> sc.getConf().get("spark.executor.decommission.forceKillTimeout",
>>>>>>> "default_value")
>>>>>>>
>>>>>>> # Print the values
>>>>>>> print(f"spark.executor.decommission.enabled: {decommission_enabled}")
>>>>>>> print(f"spark.executor.decommission.forceKillTimeout:
>>>>>>> {force_kill_timeout}")
>>>>>>>
>>>>>>> The output
>>>>>>>
>>>>>>> Spark version:  3.4.0
>>>>>>> spark.executor.decommission.enabled: true
>>>>>>> spark.executor.decommission.forceKillTimeout: 100s
>>>>>>>
>>>>>>> By creating a simple Spark application and verifying the
>>>>>>> configuration values, I trust it is shown that these two parameters are
>>>>>>> valid and are applied by Spark
>>>>>>>
>>>>>>> HTH
>>>>>>>
>>>>>>> Mich Talebzadeh,
>>>>>>>
>>>>>>> Architect | Data Engineer | Data Science | Financial Crime
>>>>>>> PhD <https://en.wikipedia.org/wiki/Doctor_of_Philosophy> Imperial
>>>>>>> College London
>>>>>>> <https://en.wikipedia.org/wiki/Imperial_College_London>
>>>>>>> London, United Kingdom
>>>>>>>
>>>>>>>
>>>>>>>    view my Linkedin profile
>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>>>>>>>
>>>>>>>
>>>>>>>  https://en.everybodywiki.com/Mich_Talebzadeh
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> *Disclaimer:* The information provided is correct to the best of my
>>>>>>> knowledge but of course cannot be guaranteed . It is essential to note
>>>>>>> that, as with any advice, quote "one test result is worth one-thousand
>>>>>>> expert opinions (Werner
>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun
>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)".
>>>>>>>
>>>>>>>
>>>>>>> On Wed, 9 Oct 2024 at 16:51, Mich Talebzadeh <
>>>>>>> mich.talebza...@gmail.com> wrote:
>>>>>>>
>>>>>>>> Do you have a better recommendation?
>>>>>>>>
>>>>>>>> Or trying to waste time as usual.
>>>>>>>>
>>>>>>>> It is far easier to throw than catch.
>>>>>>>>
>>>>>>>> Do your homework and stop throwing spanners at work.
>>>>>>>>
>>>>>>>> Mich Talebzadeh,
>>>>>>>>
>>>>>>>> Architect | Data Engineer | Data Science | Financial Crime
>>>>>>>> PhD <https://en.wikipedia.org/wiki/Doctor_of_Philosophy> Imperial
>>>>>>>> College London
>>>>>>>> <https://en.wikipedia.org/wiki/Imperial_College_London>
>>>>>>>> London, United Kingdom
>>>>>>>>
>>>>>>>>
>>>>>>>>    view my Linkedin profile
>>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>>>>>>>>
>>>>>>>>
>>>>>>>>  https://en.everybodywiki.com/Mich_Talebzadeh
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> *Disclaimer:* The information provided is correct to the best of
>>>>>>>> my knowledge but of course cannot be guaranteed . It is essential to 
>>>>>>>> note
>>>>>>>> that, as with any advice, quote "one test result is worth one-thousand
>>>>>>>> expert opinions (Werner
>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun
>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)".
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, 9 Oct 2024 at 16:43, Nicholas Chammas <
>>>>>>>> nicholas.cham...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Mich,
>>>>>>>>>
>>>>>>>>> Can you please share with the list where *exactly* you are citing
>>>>>>>>> these configs from?
>>>>>>>>>
>>>>>>>>> As far as I can tell, these two configs don’t exist and have never
>>>>>>>>> existed in the Spark codebase:
>>>>>>>>>
>>>>>>>>> spark.executor.decommission.enabled=true
>>>>>>>>> spark.executor.decommission.gracefulShutdown=true
>>>>>>>>>
>>>>>>>>> Where exactly are you getting this information from (and then
>>>>>>>>> posting it to the list as advice)? Please be clear and provide 
>>>>>>>>> specific
>>>>>>>>> references.
>>>>>>>>>
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Oct 9, 2024, at 1:20 PM, Mich Talebzadeh <
>>>>>>>>> mich.talebza...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>> Before responding, what configuration parameters are you using to
>>>>>>>>> make this work?
>>>>>>>>>
>>>>>>>>> spark.executor.decommission.enabled=true
>>>>>>>>> spark.executor.decommission.gracefulShutdown=true
>>>>>>>>> spark.executor.decommission.forceKillTimeout=100s
>>>>>>>>>
>>>>>>>>> HTH
>>>>>>>>>
>>>>>>>>> Mich Talebzadeh,
>>>>>>>>>
>>>>>>>>> Architect | Data Engineer | Data Science | Financial Crime
>>>>>>>>> PhD <https://en.wikipedia.org/wiki/Doctor_of_Philosophy> Imperial
>>>>>>>>> College London
>>>>>>>>> <https://en.wikipedia.org/wiki/Imperial_College_London>
>>>>>>>>> London, United Kingdom
>>>>>>>>>
>>>>>>>>>    view my Linkedin profile
>>>>>>>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>  https://en.everybodywiki.com/Mich_Talebzadeh
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> *Disclaimer:* The information provided is correct to the best of
>>>>>>>>> my knowledge but of course cannot be guaranteed . It is essential to 
>>>>>>>>> note
>>>>>>>>> that, as with any advice, quote "one test result is worth one-thousand
>>>>>>>>> expert opinions (Werner
>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>Von Braun
>>>>>>>>> <https://en.wikipedia.org/wiki/Wernher_von_Braun>)".
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, 9 Oct 2024 at 11:05, Jay Han <tunyu...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>> Hi spark community,
>>>>>>>>>>      I have such a question: Why driver doesn't shutdown
>>>>>>>>>> executors gracefully on k8s. For instance,
>>>>>>>>>> kubernetesClient.pods().withGracePeriod(100).delete().
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> Best,
>>>>>>>>>> Jay
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>
>> --
>>
>> Liu Cao
>>
>>
>>

Reply via email to