Key Default Type Description
state.backend.rocksdb.block.blocksize
4 kb MemorySize The approximate size (in bytes) of user data packed per block. The default blocksize is '4KB'.
state.backend.rocksdb.block.cache-size
8 mb MemorySize The amount of the cache for data blocks in RocksDB. The default block-cache size is '8MB'.
state.backend.rocksdb.block.metadata-blocksize
4 kb MemorySize Approximate size of partitioned metadata packed per block. Currently applied to indexes block when partitioned index/filters option is enabled. The default blocksize is '4KB'.
state.backend.rocksdb.bloom-filter.bits-per-key
10.0 Double Bits per key that bloom filter will use, this only take effect when bloom filter is used. The default value is 10.0.
state.backend.rocksdb.bloom-filter.block-based-mode
false Boolean If true, RocksDB will use block-based filter instead of full filter, this only take effect when bloom filter is used. The default value is 'false'.
state.backend.rocksdb.compaction.level.max-size-level-base
256 mb MemorySize The upper-bound of the total size of level base files in bytes. The default value is '256MB'.
state.backend.rocksdb.compaction.level.target-file-size-base
64 mb MemorySize The target file size for compaction, which determines a level-1 file size. The default value is '64MB'.
state.backend.rocksdb.compaction.level.use-dynamic-size
false Boolean If true, RocksDB will pick target size of each level dynamically. From an empty DB, RocksDB would make last level the base level, which means merging L0 data into the last level, until it exceeds max_bytes_for_level_base. And then repeat this process for second last level and so on. The default value is 'false'. For more information, please refer to RocksDB's doc.
state.backend.rocksdb.compaction.style
LEVEL

Enum

The specified compaction style for DB. Candidate compaction style is LEVEL, FIFO, UNIVERSAL or NONE, and Flink chooses 'LEVEL' as default style.

Possible values:
  • "LEVEL"
  • "UNIVERSAL"
  • "FIFO"
  • "NONE"
state.backend.rocksdb.compression.per.level
SNAPPY_COMPRESSION

List<Enum>

A semicolon-separated list of Compression Type. Different levels can have different compression policies. In many cases, lower levels use fast compression algorithms, while higher levels with more data use slower but more effective compression algorithms. The N th element in the List corresponds to the compression type of the level N-1When state.backend.rocksdb.compaction.level.use-dynamic-size is true, compression_per_level[0] still determines L0, but other elements are based on the base level and may not match the level seen in the info log
Note: If the List size is smaller than the level number, the undefined lower level uses the last Compression Type in the List
Some commonly used compression algorithms for candidates include NO_COMPRESSION ,SNAPPY_COMPRESSION and LZ4_COMPRESSION
The default value is SNAPPY_COMPRESSION, which means that all data uses the Snappy compression algorithm.Likewise, if set to NO_COMPRESSION , means that all data is not compressed, which will achieve faster speed but will bring some space amplification.In addition, if we need to consider both spatial amplification and performance, we can also set it to 'NO_COMPRESSION;NO_COMPRESSION;LZ4_COMPRESSION', which means that L0 and L1 data will not be compressed, and other data will be compressed using LZ4.

Possible values:
  • "NO_COMPRESSION"
  • "SNAPPY_COMPRESSION"
  • "ZLIB_COMPRESSION"
  • "BZLIB2_COMPRESSION"
  • "LZ4_COMPRESSION"
  • "LZ4HC_COMPRESSION"
  • "XPRESS_COMPRESSION"
  • "ZSTD_COMPRESSION"
  • "DISABLE_COMPRESSION_OPTION"
state.backend.rocksdb.files.open
-1 Integer The maximum number of open files (per stateful operator) that can be used by the DB, '-1' means no limit. The default value is '-1'.
state.backend.rocksdb.log.dir
(none) String The directory for RocksDB's information logging files. If empty (Flink default setting), log files will be in the same directory as the Flink log. If non-empty, this directory will be used and the data directory's absolute path will be used as the prefix of the log file name. If setting this option as a non-existing location, e.g '/dev/null', RocksDB will then create the log under its own database folder as before.
state.backend.rocksdb.log.file-num
4 Integer The maximum number of files RocksDB should keep for information logging (Default setting: 4).
state.backend.rocksdb.log.level
INFO_LEVEL

Enum

The specified information logging level for RocksDB. If unset, Flink will use INFO_LEVEL.
Note: RocksDB info logs will not be written to the TaskManager logs and there is no rolling strategy, unless you configure state.backend.rocksdb.log.dir, state.backend.rocksdb.log.max-file-size, and state.backend.rocksdb.log.file-num accordingly. Without a rolling strategy, long-running tasks may lead to uncontrolled disk space usage if configured with increased log levels!
There is no need to modify the RocksDB log level, unless for troubleshooting RocksDB.

Possible values:
  • "DEBUG_LEVEL"
  • "INFO_LEVEL"
  • "WARN_LEVEL"
  • "ERROR_LEVEL"
  • "FATAL_LEVEL"
  • "HEADER_LEVEL"
  • "NUM_INFO_LOG_LEVELS"
state.backend.rocksdb.log.max-file-size
25 mb MemorySize The maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is '25MB'.
state.backend.rocksdb.restore-overlap-fraction-threshold
0.0 Double The threshold of overlap fraction between the handle's key-group range and target key-group range. When restore base DB, only the handle which overlap fraction greater than or equal to threshold has a chance to be an initial handle. The default value is 0.0, there is always a handle will be selected for initialization.
state.backend.rocksdb.thread.num
2 Integer The maximum number of concurrent background flush and compaction jobs (per stateful operator). The default value is '2'.
state.backend.rocksdb.use-bloom-filter
false Boolean If true, every newly created SST file will contain a Bloom filter. It is disabled by default.
state.backend.rocksdb.write-batch-size
2 mb MemorySize The max size of the consumed memory for RocksDB batch write, will flush just based on item count if this config set to 0.
state.backend.rocksdb.writebuffer.count
2 Integer The maximum number of write buffers that are built up in memory. The default value is '2'.
state.backend.rocksdb.writebuffer.number-to-merge
1 Integer The minimum number of write buffers that will be merged together before writing to storage. The default value is '1'.
state.backend.rocksdb.writebuffer.size
64 mb MemorySize The amount of data built up in memory (backed by an unsorted log on disk) before converting to a sorted on-disk files. The default writebuffer size is '64MB'.