+ * Any object created with {@code new} in {@link RocksDBConfigSetter#setConfig setConfig()} and that inherits + * from {@code org.rocksdb.RocksObject} should have {@code org.rocksdb.RocksObject#close()} + * called on it here to avoid leaking off-heap memory. Objects to be closed can be saved by the user or retrieved + * back from {@code options} using its getter methods. + *
+ * Example objects needing to be closed include {@code org.rocksdb.Filter} and {@code org.rocksdb.Cache}.
+ *
+ * @param storeName the name of the store being configured
+ * @param options the RocksDB options
+ */
+ default void close(final String storeName, final Options options) {
+ LOG.warn("The default close will be removed in 3.0.0 -- you should overwrite it if you have implemented RocksDBConfigSetter");
+ }
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
index e4c416c231b..ed744686c2c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
@@ -92,6 +92,8 @@ public class RocksDBStore implements KeyValueStore