From c4f4f2a8a13684a8899c218a39ad931676b92031 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 12 Jun 2026 17:30:01 -0700 Subject: [PATCH] KAFKA-20688: Fix RocksDB memory leak --- .../state/internals/RocksDBTimestampedStoreWithHeaders.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStoreWithHeaders.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStoreWithHeaders.java index 9a53c35290a3e..8f96b18291652 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStoreWithHeaders.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStoreWithHeaders.java @@ -73,7 +73,7 @@ void openRocksDB(final DBOptions dbOptions, final ColumnFamilyOptions columnFamilyOptions) { // Check if we're upgrading from RocksDBTimestampedStore or from plain RocksDBStore final List existingCFs; - try (final Options options = new Options(dbOptions, new ColumnFamilyOptions())) { + try (final Options options = new Options(dbOptions, columnFamilyOptions)) { existingCFs = RocksDB.listColumnFamilies(options, dbDir.getAbsolutePath()); } catch (final RocksDBException e) { throw new ProcessorStateException("Error listing column families for store " + name, e);