@PublicEvolving public interface RocksDBOptionsFactory extends Serializable
DBOptions
and ColumnFamilyOptions
to be passed to the EmbeddedRocksDBStateBackend
. Options have to be created lazily by this factory, because the
Options
class is not serializable and holds pointers to native code.
A typical pattern to use this RocksDBOptionsFactory is as follows:
rocksDbBackend.setRocksDBOptions(new RocksDBOptionsFactory() {
public DBOptions createDBOptions(DBOptions currentOptions, Collection<AutoCloseable> handlesToClose) {
return currentOptions.setMaxOpenFiles(1024);
}
public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions currentOptions, Collection<AutoCloseable> handlesToClose) {
BloomFilter bloomFilter = new BloomFilter();
handlesToClose.add(bloomFilter);
return currentOptions
.setTableFormatConfig(
new BlockBasedTableConfig().setFilter(bloomFilter));
}
});
Modifier and Type | Method and Description |
---|---|
org.rocksdb.ColumnFamilyOptions |
createColumnOptions(org.rocksdb.ColumnFamilyOptions currentOptions,
Collection<AutoCloseable> handlesToClose)
This method should set the additional options on top of the current options object.
|
org.rocksdb.DBOptions |
createDBOptions(org.rocksdb.DBOptions currentOptions,
Collection<AutoCloseable> handlesToClose)
This method should set the additional options on top of the current options object.
|
default RocksDBNativeMetricOptions |
createNativeMetricsOptions(RocksDBNativeMetricOptions nativeMetricOptions)
This method should enable certain RocksDB metrics to be forwarded to Flink's metrics
reporter.
|
default org.rocksdb.ReadOptions |
createReadOptions(org.rocksdb.ReadOptions currentOptions,
Collection<AutoCloseable> handlesToClose)
This method should set the additional options on top of the current options object.
|
default org.rocksdb.WriteOptions |
createWriteOptions(org.rocksdb.WriteOptions currentOptions,
Collection<AutoCloseable> handlesToClose)
This method should set the additional options on top of the current options object.
|
org.rocksdb.DBOptions createDBOptions(org.rocksdb.DBOptions currentOptions, Collection<AutoCloseable> handlesToClose)
It is important to set the options on the current object and return the result from the setter methods, otherwise the pre-defined options may get lost.
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly created RocksObject
s.org.rocksdb.ColumnFamilyOptions createColumnOptions(org.rocksdb.ColumnFamilyOptions currentOptions, Collection<AutoCloseable> handlesToClose)
It is important to set the options on the current object and return the result from the setter methods, otherwise the pre-defined options may get lost.
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly created RocksObject
s.default RocksDBNativeMetricOptions createNativeMetricsOptions(RocksDBNativeMetricOptions nativeMetricOptions)
Enabling these monitoring options may degrade RockDB performance and should be set with care.
nativeMetricOptions
- The options object with the pre-defined options.default org.rocksdb.WriteOptions createWriteOptions(org.rocksdb.WriteOptions currentOptions, Collection<AutoCloseable> handlesToClose)
It is important to set the options on the current object and return the result from the setter methods, otherwise the pre-defined options may get lost.
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly created RocksObject
s.default org.rocksdb.ReadOptions createReadOptions(org.rocksdb.ReadOptions currentOptions, Collection<AutoCloseable> handlesToClose)
It is important to set the options on the current object and return the result from the setter methods, otherwise the pre-defined options may get lost.
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly created RocksObject
s.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.