aliehsaeedii commented on code in PR #14626:
URL: https://github.com/apache/kafka/pull/14626#discussion_r1404613145


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/LogicalKeyValueSegment.java:
##########
@@ -170,7 +173,30 @@ public boolean isOpen() {
 
     @Override
     public synchronized byte[] get(final Bytes key) {
-        return physicalStore.get(prefixKeyFormatter.addPrefix(key));
+        return get(key, Optional.empty());
+    }
+
+    public synchronized byte[] get(final Bytes key, final Snapshot snapshot) {
+        return get(key, Optional.of(snapshot));
+    }
+
+    private synchronized byte[] get(final Bytes key, final Optional<Snapshot> 
snapshot) {
+        if (snapshot.isPresent()) {
+            try (ReadOptions readOptions = new ReadOptions()) {
+                readOptions.setSnapshot(snapshot.get());
+                return physicalStore.get(prefixKeyFormatter.addPrefix(key), 
readOptions);
+            }
+        } else {
+            return physicalStore.get(prefixKeyFormatter.addPrefix(key));
+        }
+    }
+
+    public Snapshot getSnapshot() {
+        return physicalStore.db.getSnapshot();

Review Comment:
   > I think we should push this into `RocksDBStore` class, ie, call 
`physicalStore.getSnapshot()` here, and extend `RocksDBStore` to track all open 
snapshots (similar to how we track open iterators and release all open snapshot 
if the store is closed).
   
   Since "Snapshots do not persist across DB restarts 
([ref](https://github.com/facebook/rocksdb/wiki/Snapshot))", I think we do not 
need to track them when closing the store. 



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to