Interface RocksDBOptionsFactory
-
- All Superinterfaces:
Serializable
- All Known Subinterfaces:
ConfigurableRocksDBOptionsFactory
,ConfigurableRocksDBOptionsFactory
,RocksDBOptionsFactory
@PublicEvolving public interface RocksDBOptionsFactory extends Serializable
A factory forDBOptions
andColumnFamilyOptions
to be passed to theEmbeddedRocksDBStateBackend
. Options have to be created lazily by this factory, because theOptions
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)); } });
-
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Modifier and Type Method 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.
-
-
-
Method Detail
-
createDBOptions
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. The current options object may contain pre-defined options based on flags that have been configured on the state backend.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.
- Parameters:
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly createdRocksObject
s.- Returns:
- The options object on which the additional options are set.
-
createColumnOptions
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. The current options object may contain pre-defined options based on flags that have been configured on the state backend.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.
- Parameters:
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly createdRocksObject
s.- Returns:
- The options object on which the additional options are set.
-
createNativeMetricsOptions
default RocksDBNativeMetricOptions createNativeMetricsOptions(RocksDBNativeMetricOptions nativeMetricOptions)
This method should enable certain RocksDB metrics to be forwarded to Flink's metrics reporter.Enabling these monitoring options may degrade RockDB performance and should be set with care.
- Parameters:
nativeMetricOptions
- The options object with the pre-defined options.- Returns:
- The options object on which the additional options are set.
-
createWriteOptions
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. The current options object may contain pre-defined options based on flags that have been configured on the state backend.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.
- Parameters:
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly createdRocksObject
s.- Returns:
- The options object on which the additional options are set.
-
createReadOptions
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. The current options object may contain pre-defined options based on flags that have been configured on the state backend.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.
- Parameters:
currentOptions
- The options object with the pre-defined options.handlesToClose
- The collection to register newly createdRocksObject
s.- Returns:
- The options object on which the additional options are set.
-
-