Re: Apache Spark 3.4.3 (?)

2024-04-06 Thread Mridul Muralidharan
Hi Dongjoon,

  Thanks for volunteering !
I would suggest to wait for SPARK-47318 to be merged as well for 3.4

Regards,
Mridul

On Sat, Apr 6, 2024 at 6:49 PM Dongjoon Hyun 
wrote:

> Hi, All.
>
> Apache Spark 3.4.2 tag was created on Nov 24th and `branch-3.4` has 85
> commits including important security and correctness patches like
> SPARK-45580, SPARK-46092, SPARK-46466, SPARK-46794, and SPARK-46862.
>
> https://github.com/apache/spark/releases/tag/v3.4.2
>
> $ git log --oneline v3.4.2..HEAD | wc -l
>   85
>
> SPARK-45580 Subquery changes the output schema of the outer query
> SPARK-46092 Overflow in Parquet row group filter creation causes incorrect
> results
> SPARK-46466 Vectorized parquet reader should never do rebase for timestamp
> ntz
> SPARK-46794 Incorrect results due to inferred predicate from checkpoint
> with subquery
> SPARK-46862 Incorrect count() of a dataframe loaded from CSV datasource
> SPARK-45445 Upgrade snappy to 1.1.10.5
> SPARK-47428 Upgrade Jetty to 9.4.54.v20240208
> SPARK-46239 Hide `Jetty` info
>
>
> Currently, I'm checking more applicable patches for branch-3.4. I'd like
> to propose to release Apache Spark 3.4.3 and volunteer as the release
> manager for Apache Spark 3.4.3. If there are no additional blockers, the
> first tentative RC1 vote date is April 15th (Monday).
>
> WDYT?
>
>
> Dongjoon.
>


Re: Apache Spark 3.4.3 (?)

2024-04-06 Thread Holden Karau
Sounds good to me :)

Twitter: https://twitter.com/holdenkarau
Books (Learning Spark, High Performance Spark, etc.):
https://amzn.to/2MaRAG9  
YouTube Live Streams: https://www.youtube.com/user/holdenkarau


On Sat, Apr 6, 2024 at 2:51 PM Dongjoon Hyun 
wrote:

> Hi, All.
>
> Apache Spark 3.4.2 tag was created on Nov 24th and `branch-3.4` has 85
> commits including important security and correctness patches like
> SPARK-45580, SPARK-46092, SPARK-46466, SPARK-46794, and SPARK-46862.
>
> https://github.com/apache/spark/releases/tag/v3.4.2
>
> $ git log --oneline v3.4.2..HEAD | wc -l
>   85
>
> SPARK-45580 Subquery changes the output schema of the outer query
> SPARK-46092 Overflow in Parquet row group filter creation causes incorrect
> results
> SPARK-46466 Vectorized parquet reader should never do rebase for timestamp
> ntz
> SPARK-46794 Incorrect results due to inferred predicate from checkpoint
> with subquery
> SPARK-46862 Incorrect count() of a dataframe loaded from CSV datasource
> SPARK-45445 Upgrade snappy to 1.1.10.5
> SPARK-47428 Upgrade Jetty to 9.4.54.v20240208
> SPARK-46239 Hide `Jetty` info
>
>
> Currently, I'm checking more applicable patches for branch-3.4. I'd like
> to propose to release Apache Spark 3.4.3 and volunteer as the release
> manager for Apache Spark 3.4.3. If there are no additional blockers, the
> first tentative RC1 vote date is April 15th (Monday).
>
> WDYT?
>
>
> Dongjoon.
>


Re: Vote on Dynamic resource allocation for structured streaming [SPARK-24815]

2024-04-06 Thread Pavan Kotikalapudi
Hi Jungtaek,

Status on current SPARK-24815
:
Thomas Graves is reviewing the draft PR
. I need to add documentation
about the configs and usage details, I am planning to do that this week.
He did mention that it would be great if somebody with experience in
structured streaming would take a look at the algorithm. Will you be able
to review it?

Another point I wanted to discuss is, as you might have already seen
in the design
doc

we
use traditional DRA configs
spark.dynamicAllocation.enabled,
spark.dynamicAllocation.schedulerBacklogTimeout,
spark.dynamicAllocation.sustainedSchedulerBacklogTimeout,
spark.dynamicAllocation.executorIdleTimeout,
spark.dynamicAllocation.cachedExecutorIdleTimeout,

and few new configs

spark.dynamicAllocation.streaming.enabled,
spark.dynamicAllocation.streaming.executorDeallocationRatio,
spark.dynamicAllocation.streaming.executorDeallocationTimeout.

to make the DRA work for structured streaming.

While in the design doc I did mention that we have to calculate  and set
scale out/back thresholds based on the trigger interval
.
We (internally in the company) do have helper functions to auto-generate
the above configs based on trigger interval and the threshold configs (we
also got similar feedback in reviews

).
Here are such configs

  # required - should be greater than 3 seconds as that gives enough
seconds for scaleOut and scaleBack thresholds to work with.
  "spark.sql.streaming.triggerInterval.seconds": 
  # optional - value should be between 0 and 1 and greater than
scaleBackThreshold : default is 0.9
  "spark.dynamicAllocation.streaming.scaleOutThreshold": 
  # optional - value should be between 0 and 1 and less than
scaleOutThreshold : default is 0.5
  "spark.dynamicAllocation.streaming.scaleBackThreshold": 

The above configs helps us to generate the below configs for app with
different trigger intervals ( or if they change them for some reason)

spark.dynamicAllocation.schedulerBacklogTimeout,
spark.dynamicAllocation.sustainedSchedulerBacklogTimeout,
spark.dynamicAllocation.executorIdleTimeout,
spark.dynamicAllocation.cachedExecutorIdleTimeout.

While our additional configs have its own limitations. I would like to get
some feedback if adding such kinds of configs to automate
the process of calculating the thresholds and their respective configs
makes sense?

Thank you,

Pavan

On Thu, Mar 28, 2024 at 3:38 PM Pavan Kotikalapudi 
wrote:

> Hi Jungtaek,
>
> Sorry for the late reply.
>
> I understand the concerns towards finding PMC members, I had similar
> concerns in the past. Do you think we have something to improve in the SPIP
> (certain areas) so that it would get traction from PMC members? Or this
> SPIP might not be a priority to the PMC right now?
>
> I agree this change is small enough that it might not be tagged as an
> SPIP. I started with the template SPIP questions so that it would be easier
> to understand the limitations of the current system, new solution, how it
> works, how to use it, limitations etcAs you might have already
> noticed in the PR, This change is turned off by default, will only work if
> `spark.dynamicAllocation.streaming.enabled` is true.
>
> Regarding the concerns about expertise in DRA,  I will find some core
> contributors of this module/DRA and tag them to this email with details,
> Mich has also highlighted the same in the past. Once we get approval from
> them we can further discuss and enhance this to make the user experience
> better.
>
> Thank you,
>
> Pavan
>
>
> On Tue, Mar 26, 2024 at 8:12 PM Jungtaek Lim 
> wrote:
>
>> Sounds good.
>>
>> One thing I'd like to clarify before shepherding this SPIP is the process
>> itself. Getting enough traction from PMC members is another issue to pass
>> the SPIP vote. Even a vote from committer is not counted. (I don't have a
>> binding vote.) I only see one PMC member (Thomas Graves, not my team) in
>> the design doc and we still don't get positive feedback. So still a long
>> way to go. We need three supporters from PMC members.
>>
>> Another thing is, I get the proposal at a high level, but I don't have
>> actual expertise in DRA. I could review the code in general, but I feel
>> like I'm not qualified to approve the code. We still need an expert on the
>> CORE area, especially who has expertise with DRA. (Could you please
>> annotate the code and enumerate several people who worked on the codebase?)
>> If they need an expertise of streaming to understand how things will work
>> then either you or I can explain, but I can't just approve and merge the
>> code.

Re: External Spark shuffle service for k8s

2024-04-06 Thread Mich Talebzadeh
Thanks for your suggestion that I take it as a workaround. Whilst this
workaround can potentially address storage allocation issues, I was more
interested in exploring solutions that offer a more seamless integration
with large distributed file systems like HDFS, GCS, or S3. This would
ensure better performance and scalability for handling larger datasets
efficiently.


Mich Talebzadeh,
Technologist | Solutions Architect | Data Engineer  | Generative AI
London
United Kingdom


   view my Linkedin profile



 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  Von
Braun )".


On Sat, 6 Apr 2024 at 21:28, Bjørn Jørgensen 
wrote:

> You can make a PVC on K8S call it 300GB
>
> make a folder in yours dockerfile
> WORKDIR /opt/spark/work-dir
> RUN chmod g+w /opt/spark/work-dir
>
> start spark with adding this
>
> .config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.options.claimName",
> "300gb") \
>
> .config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.mount.path",
> "/opt/spark/work-dir") \
>
> .config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.mount.readOnly",
> "False") \
>
> .config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.options.claimName",
> "300gb") \
>
> .config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.mount.path",
> "/opt/spark/work-dir") \
>
> .config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.mount.readOnly",
> "False") \
>   .config("spark.local.dir", "/opt/spark/work-dir")
>
>
>
>
> lør. 6. apr. 2024 kl. 15:45 skrev Mich Talebzadeh <
> mich.talebza...@gmail.com>:
>
>> I have seen some older references for shuffle service for k8s,
>> although it is not clear they are talking about a generic shuffle
>> service for k8s.
>>
>> Anyhow with the advent of genai and the need to allow for a larger
>> volume of data, I was wondering if there has been any more work on
>> this matter. Specifically larger and scalable file systems like HDFS,
>> GCS , S3 etc, offer significantly larger storage capacity than local
>> disks on individual worker nodes in a k8s cluster, thus allowing
>> handling much larger datasets more efficiently. Also the degree of
>> parallelism and fault tolerance  with these files systems come into
>> it. I will be interested in hearing more about any progress on this.
>>
>> Thanks
>> .
>>
>> Mich Talebzadeh,
>>
>> Technologist | Solutions Architect | Data Engineer  | Generative AI
>>
>> London
>> United Kingdom
>>
>>
>>view my Linkedin profile
>>
>>
>>  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 Von Braun)".
>>
>> -
>> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>>
>>
>
> --
> Bjørn Jørgensen
> Vestre Aspehaug 4, 6010 Ålesund
> Norge
>
> +47 480 94 297
>


Apache Spark 3.4.3 (?)

2024-04-06 Thread Dongjoon Hyun
Hi, All.

Apache Spark 3.4.2 tag was created on Nov 24th and `branch-3.4` has 85
commits including important security and correctness patches like
SPARK-45580, SPARK-46092, SPARK-46466, SPARK-46794, and SPARK-46862.

https://github.com/apache/spark/releases/tag/v3.4.2

$ git log --oneline v3.4.2..HEAD | wc -l
  85

SPARK-45580 Subquery changes the output schema of the outer query
SPARK-46092 Overflow in Parquet row group filter creation causes incorrect
results
SPARK-46466 Vectorized parquet reader should never do rebase for timestamp
ntz
SPARK-46794 Incorrect results due to inferred predicate from checkpoint
with subquery
SPARK-46862 Incorrect count() of a dataframe loaded from CSV datasource
SPARK-45445 Upgrade snappy to 1.1.10.5
SPARK-47428 Upgrade Jetty to 9.4.54.v20240208
SPARK-46239 Hide `Jetty` info


Currently, I'm checking more applicable patches for branch-3.4. I'd like to
propose to release Apache Spark 3.4.3 and volunteer as the release manager
for Apache Spark 3.4.3. If there are no additional blockers, the first
tentative RC1 vote date is April 15th (Monday).

WDYT?

Dongjoon.


Re: External Spark shuffle service for k8s

2024-04-06 Thread Bjørn Jørgensen
You can make a PVC on K8S call it 300GB

make a folder in yours dockerfile
WORKDIR /opt/spark/work-dir
RUN chmod g+w /opt/spark/work-dir

start spark with adding this

.config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.options.claimName",
"300gb") \

.config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.mount.path",
"/opt/spark/work-dir") \

.config("spark.kubernetes.driver.volumes.persistentVolumeClaim.300gb.mount.readOnly",
"False") \

.config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.options.claimName",
"300gb") \

.config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.mount.path",
"/opt/spark/work-dir") \

.config("spark.kubernetes.executor.volumes.persistentVolumeClaim.300gb.mount.readOnly",
"False") \
  .config("spark.local.dir", "/opt/spark/work-dir")




lør. 6. apr. 2024 kl. 15:45 skrev Mich Talebzadeh :

> I have seen some older references for shuffle service for k8s,
> although it is not clear they are talking about a generic shuffle
> service for k8s.
>
> Anyhow with the advent of genai and the need to allow for a larger
> volume of data, I was wondering if there has been any more work on
> this matter. Specifically larger and scalable file systems like HDFS,
> GCS , S3 etc, offer significantly larger storage capacity than local
> disks on individual worker nodes in a k8s cluster, thus allowing
> handling much larger datasets more efficiently. Also the degree of
> parallelism and fault tolerance  with these files systems come into
> it. I will be interested in hearing more about any progress on this.
>
> Thanks
> .
>
> Mich Talebzadeh,
>
> Technologist | Solutions Architect | Data Engineer  | Generative AI
>
> London
> United Kingdom
>
>
>view my Linkedin profile
>
>
>  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 Von Braun)".
>
> -
> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>
>

-- 
Bjørn Jørgensen
Vestre Aspehaug 4, 6010 Ålesund
Norge

+47 480 94 297


External Spark shuffle service for k8s

2024-04-06 Thread Mich Talebzadeh
I have seen some older references for shuffle service for k8s,
although it is not clear they are talking about a generic shuffle
service for k8s.

Anyhow with the advent of genai and the need to allow for a larger
volume of data, I was wondering if there has been any more work on
this matter. Specifically larger and scalable file systems like HDFS,
GCS , S3 etc, offer significantly larger storage capacity than local
disks on individual worker nodes in a k8s cluster, thus allowing
handling much larger datasets more efficiently. Also the degree of
parallelism and fault tolerance  with these files systems come into
it. I will be interested in hearing more about any progress on this.

Thanks
.

Mich Talebzadeh,

Technologist | Solutions Architect | Data Engineer  | Generative AI

London
United Kingdom


   view my Linkedin profile


 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 Von Braun)".

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