schenksj opened a new pull request, #43202:
URL: https://github.com/apache/spark/pull/43202

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: 
https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: 
https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., 
'[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a 
faster review.
     7. If you want to add a new configuration, please read the guideline first 
for naming configurations in
        
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the 
guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   Our spark environment features a number of parallel structured streaming 
jobs, many of which have use state store.  Most use state store for 
dropDuplicates and work with a tiny amount of information, but a few have a 
substantially large state store requiring use of RocksDB.  In such a 
configuration, spark allocates a minimum of `spark.sql.shuffle.partitions * 
queryCount` partitions, each of which pre-allocate about 74mb (observed on 
EMR/Hadoop) disk storage for RocksDB.  This allocation is due to pre-allocation 
of log files space using 
[fallocate](https://github.com/facebook/rocksdb/blob/main/include/rocksdb/options.h#L871-L880),
 requiring users to either unnaturally reduce shuffle partitions, split running 
spark instances, or allocating a large amount of wasted storage.
   
   This PR provides users with the option to simply disable fallocate so 
RocksDB uses far less space for the smaller state stores, reducing complexity 
and disk storage at the expense of performance.
   
   <!--
   Please clarify what changes you are proposing. The purpose of this section 
is to outline the changes and how this PR fixes the issue.
   If possible, please consider writing useful notes for better and faster 
reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class 
hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other 
DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   As previously mentioned, these changes allow a spark context to support many 
parallel structured streaming jobs when using RocksDB state stores without the 
need to allocate a glut of excess storage.
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as 
the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to 
the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Users disable the fallocate rocksdb performance optimization by configuring 
`spark.sql.streaming.stateStore.rocksdb.allowFAllocate=false`
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions 
for the consistent environment, and the instructions could accord to: 
https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   1) A few test cases were added
   2) The state store size was validated by running this script with and 
without fallocate disabled
   
   ```
   from pyspark.sql.types import StructType, StructField, StringType, 
TimestampType
   import datetime
   
   if disable_fallocate:
      spark.conf.set("spark.sql.streaming.stateStore.rocksdb.allowFAllocate", 
"false")
   
   spark.conf.set(
       "spark.sql.streaming.stateStore.providerClass",
       
"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider",
   )
   
   
   schema = StructType(
       [
           StructField("one", TimestampType(), False),
           StructField("two", StringType(), True),
       ]
   )
   
   now = datetime.datetime.now()
   data = [(now, y) for y in range(300)]
   init_df = spark.createDataFrame(data, schema)
   
   path = "/tmp/stream_try/test"
   init_df.write.format("parquet").mode("append").save(path)
   
   stream_df = spark.readStream.schema(schema).format("parquet").load(path)
   
   stream_df = stream_df.dropDuplicates(["one"])
   
   
   def foreach_batch_function(batch_df, epoch_id):
       batch_df.write.format("parquet").mode("append").option("path", path + 
"_out").save()
   
   
   stream_df.writeStream.foreachBatch(foreach_batch_function).option(
       "checkpointLocation", path + "_checkpoint"
   ).start()
   ```
   
   With these results (local run, docker container with small FS)
   ```
   allowFAllocate=True (current default)
   ---------------------
   root@0ef384f699e0:/tmp# du -sh spark-d43a2964-c92a-4d94-9fdd-f3557a651fd9
   808M    spark-d43a2964-c92a-4d94-9fdd-f3557a651fd9
   |
   |-->4.1M        
StateStoreId(opId=0,partId=0,name=default)-d59b907c-8004-47f9-a8a1-dec131f73505
   |--> <snip>
   |-->4.1M        
StateStoreId(opId=0,partId=199,name=default)-b49a93fe-1007-4e92-8f8f-5767aef41e5c
   
   allowFAllocate=False (new feature)
   ----------------------
   root@0ef384f699e0:/tmp# du -sh spark-00cb768d-2659-453c-8670-4aaf70148041
   
   7.9M    spark-00cb768d-2659-453c-8670-4aaf70148041
   |
   |-->40K StateStoreId(opId=0,partId=0,name=default)-45b38d9c-737b-49b1-bb82-
   |--> <snip>
   |-->40K 
StateStoreId(opId=0,partId=199,name=default)-28a6cc02-2693-4360-b47a-1f1ab0d54a61
   ```
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this 
patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling 
Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   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