rkhachatryan commented on a change in pull request #16171:
URL: https://github.com/apache/flink/pull/16171#discussion_r654939097
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
##########
@@ -497,21 +507,25 @@ private RocksDBRestoreOperation
getRocksDBRestoreOperation(
long lastCompletedCheckpointId) {
RocksDBSnapshotStrategyBase<K, ?> checkpointSnapshotStrategy;
if (enableIncrementalCheckpointing) {
+ RocksDBStateUploader stateUploader =
+ injectRocksDBStateUploader == null
+ ? new
RocksDBStateUploader(numberOfTransferingThreads)
+ : injectRocksDBStateUploader;
checkpointSnapshotStrategy =
new RocksIncrementalSnapshotStrategy<>(
- db,
- rocksDBResourceGuard,
- keySerializerProvider.currentSchemaSerializer(),
- kvStateInformation,
- keyGroupRange,
- keyGroupPrefixBytes,
- localRecoveryConfig,
- cancelStreamRegistry,
- instanceBasePath,
- backendUID,
- materializedSstFiles,
- lastCompletedCheckpointId,
- numberOfTransferingThreads);
+ db,
+ rocksDBResourceGuard,
+
keySerializerProvider.currentSchemaSerializer(),
+ kvStateInformation,
+ keyGroupRange,
+ keyGroupPrefixBytes,
+ localRecoveryConfig,
+ cancelStreamRegistry,
+ instanceBasePath,
+ backendUID,
+ materializedSstFiles,
+ lastCompletedCheckpointId)
+ .setRocksDBStateUploader(stateUploader);
Review comment:
I think there is no need to introduce a setter in
`RocksIncrementalSnapshotStrategy`.
`stateUploader` can be passed to its constructor and be a final field there
(in fact, I think one of the benefits of having a builder is immutability of
the object it builds).
WDYT?
##########
File path:
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackendBuilder.java
##########
@@ -238,6 +242,12 @@ public RocksDBKeyedStateBackendBuilder(
return this;
}
+ RocksDBKeyedStateBackendBuilder<K> setRocksDBStateUploader(
+ RocksDBStateUploader rocksDBStateUploader) {
+ this.injectRocksDBStateUploader = rocksDBStateUploader;
Review comment:
Should we also have a similar here as in `setNumberOfTransferingThreads`?
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]