xi-db opened a new pull request, #52894:
URL: https://github.com/apache/spark/pull/52894

   ### What changes were proposed in this pull request?
   
   Currently, Spark Connect enforce gRPC message limits on both the client and 
the server. These limits are largely meant to protect the server from potential 
OOMs by rejecting abnormally large messages. However, there are several cases 
where genuine messages exceed the limit and cause execution failures.
   
   To improve Spark Connect stability, this PR implements compressing 
unresolved proto plans to mitigate the issue of oversized messages from the 
client to the server. The compression applies to ExecutePlan and AnalyzePlan - 
the only two methods that might hit the message limit. The other issue of 
message limit from the server to the client is a different issue, and it’s out 
of the scope (that one is already fixed in 
https://github.com/apache/spark/pull/52271).
   
   (Scala client changes are being implemented in a follow-up PR.)
   
   To reproduce the existing issue we are solving here, run this code on Spark 
Connect:
   
   ```
   import random
   import string
   
   def random_letters(length: int) -> str:
       return ''.join(random.choices(string.ascii_letters, k=length))
   
   num_unique_small_relations = 5
   size_per_small_relation = 512 * 1024
   small_dfs = 
[spark.createDataFrame([(random_letters(size_per_small_relation),)],) for _ in 
range(num_unique_small_relations)]
   result_df = small_dfs[0]
   for _ in range(512):
       result_df = result_df.unionByName(small_dfs[random.randint(0, 
len(small_dfs) - 1)])
   result_df.collect()
   ```
   
   It fails with `StatusCode.RESOURCE_EXHAUSTED` error with message`Sent 
message larger than max (269178955 vs. 134217728)`, because the client was 
trying to send a too large message to the server.
   
   Note: repeated small local relations is only one way causing a large plan, 
the size of the plan can also be contributed by repeated subtrees of plan 
transformations, serialized UDFs, captured external variables by UDFs, etc.
   
   With the improvement introduced by the PR, the above code runs successfully 
and prints the expected result.
   
   ### Why are the changes needed?
   
   It improves Spark Connect stability when executing and analyzing large plans.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   New tests on both the server side and the client side.
   
   ### Was this patch authored or co-authored using generative AI tooling?
   
   No.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to