This is an automated email from the ASF dual-hosted git repository. kabhwan pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new b573cca90ea [SPARK-44288][SS] Set the column family options before passing to DBOptions in RocksDB state store provider b573cca90ea is described below commit b573cca90ea843f8b492c5b1a72463854d1568c2 Author: Anish Shrigondekar <anish.shrigonde...@databricks.com> AuthorDate: Tue Jul 4 13:06:32 2023 +0900 [SPARK-44288][SS] Set the column family options before passing to DBOptions in RocksDB state store provider ### What changes were proposed in this pull request? Set the column family options before passing to DBOptions in RocksDB state store provider ### Why are the changes needed? Address bug fix to ensure column family options around memory usage are passed correctly to dbOptions ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests Closes #41840 from anishshri-db/task/SPARK-44288. Authored-by: Anish Shrigondekar <anish.shrigonde...@databricks.com> Signed-off-by: Jungtaek Lim <kabhwan.opensou...@gmail.com> --- .../org/apache/spark/sql/execution/streaming/state/RocksDB.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index a9c15cf7f7d..65299ea37ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -92,9 +92,6 @@ class RocksDB( private val columnFamilyOptions = new ColumnFamilyOptions() - private val dbOptions = - new Options(new DBOptions(), columnFamilyOptions) // options to open the RocksDB - // Set RocksDB options around MemTable memory usage. By default, we let RocksDB // use its internal default values for these settings. if (conf.writeBufferSizeMB > 0L) { @@ -105,6 +102,9 @@ class RocksDB( columnFamilyOptions.setMaxWriteBufferNumber(conf.maxWriteBufferNumber) } + private val dbOptions = + new Options(new DBOptions(), columnFamilyOptions) // options to open the RocksDB + dbOptions.setCreateIfMissing(true) dbOptions.setTableFormatConfig(tableFormatConfig) dbOptions.setMaxOpenFiles(conf.maxOpenFiles) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org