Skip to content

HDDS-13860. RocksDatabase#open leaks column family handles when failing to read cfOptions from file#9225

Merged
smengcl merged 1 commit intoapache:masterfrom
smengcl:HDDS-13860-fix-leak
Oct 31, 2025
Merged

HDDS-13860. RocksDatabase#open leaks column family handles when failing to read cfOptions from file#9225
smengcl merged 1 commit intoapache:masterfrom
smengcl:HDDS-13860-fix-leak

Conversation

@smengcl
Copy link
Contributor

@smengcl smengcl commented Oct 31, 2025

What changes were proposed in this pull request?

(Found this during SnapshotDefragService development, but this bug is not in SDS.)

In RocksDatabase#open, when it fails to read column family options from file, it will fall back to getting the default column family options:

if (cfOptions == null) {
cfOptions = DBStoreBuilder.HDDS_DEFAULT_DB_PROFILE.getColumnFamilyOptions();
}

Corresponding error messages when failing to read cfOptions from file:

2025-10-30 13:06:49,981 [om1-SnapshotDefragService#0] ERROR db.DBStoreBuilder (DBStoreBuilder.java:getCfOptionsFromFile(440)) - Error while trying to read ColumnFamilyOptions from file: /path/to/ozone/hadoop-ozone/integration-test/target/test-dir/MiniOzoneClusterImpl-56d84c7d-ccb4-4e14-b881-7bd22db88684/om/data/db.snapshots/checkpointStateDefragged
2025-10-30 13:06:49,981 [om1-SnapshotDefragService#0] ERROR db.DBStoreBuilder (DBStoreBuilder.java:getDefaultDBOptions(397)) - Error trying to use dbOptions from file: null

But those options aren't properly released later, leading to leaks, e.g. output:

java.lang.AssertionError: Found 19 leaked objects, check logs

	at org.apache.hadoop.hdds.utils.db.managed.ManagedRocksObjectMetrics.assertNoLeaks(ManagedRocksObjectMetrics.java:54)
	at org.apache.hadoop.ozone.MiniOzoneClusterImpl.shutdown(MiniOzoneClusterImpl.java:381)
...
2025-10-30 19:37:10,794 [LeakDetector-ManagedRocksObject0] WARN  managed.ManagedRocksObjectUtils (HddsUtils.java:reportLeak(902)) - ManagedColumnFamilyOptions is not closed properly
StackTrace for unclosed instance: org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions.<init>(ManagedColumnFamilyOptions.java:36)
org.apache.hadoop.hdds.utils.db.DBProfile$1.getColumnFamilyOptions(DBProfile.java:51)
org.apache.hadoop.hdds.utils.db.DBProfile$2.getColumnFamilyOptions(DBProfile.java:112)
org.apache.hadoop.hdds.utils.db.TableConfig.newTableConfig(TableConfig.java:59)
org.apache.hadoop.hdds.utils.db.RocksDatabase.lambda$getExtraColumnFamilies$1(RocksDatabase.java:133)
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177)
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
org.apache.hadoop.hdds.utils.db.RocksDatabase.getExtraColumnFamilies(RocksDatabase.java:134)
org.apache.hadoop.hdds.utils.db.RocksDatabase.open(RocksDatabase.java:164)
org.apache.hadoop.hdds.utils.db.RDBStore.<init>(RDBStore.java:111)
org.apache.hadoop.hdds.utils.db.DBStoreBuilder.build(DBStoreBuilder.java:227)
org.apache.hadoop.ozone.om.SnapshotDefragService.processCheckpointDb(SnapshotDefragService.java:226)
org.apache.hadoop.ozone.om.SnapshotDefragService.performFullDefragmentation(SnapshotDefragService.java:311)
org.apache.hadoop.ozone.om.SnapshotDefragService.triggerSnapshotDefragOnce(SnapshotDefragService.java:1026)
org.apache.hadoop.ozone.om.SnapshotDefragService$SnapshotDefragTask.call(SnapshotDefragService.java:942)
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask.lambda$run$0(BackgroundService.java:171)
java.base/java.util.concurrent.CompletableFuture$AsyncRun.run$$$capture(CompletableFuture.java:1736)
java.base/java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java)
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:829)

2025-10-30 19:37:10,794 [LeakDetector-ManagedRocksObject0] WARN  managed.ManagedRocksObjectUtils (HddsUtils.java:reportLeak(902)) - ManagedColumnFamilyOptions is not closed properly
StackTrace for unclosed instance: org.apache.hadoop.hdds.utils.db.managed.ManagedColumnFamilyOptions.<init>(ManagedColumnFamilyOptions.java:36)
org.apache.hadoop.hdds.utils.db.DBProfile$1.getColumnFamilyOptions(DBProfile.java:51)
org.apache.hadoop.hdds.utils.db.DBProfile$2.getColumnFamilyOptions(DBProfile.java:112)
org.apache.hadoop.hdds.utils.db.TableConfig.newTableConfig(TableConfig.java:59)
org.apache.hadoop.hdds.utils.db.RocksDatabase.lambda$getExtraColumnFamilies$1(RocksDatabase.java:133)
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177)
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
org.apache.hadoop.hdds.utils.db.RocksDatabase.getExtraColumnFamilies(RocksDatabase.java:134)
org.apache.hadoop.hdds.utils.db.RocksDatabase.open(RocksDatabase.java:164)
org.apache.hadoop.hdds.utils.db.RDBStore.<init>(RDBStore.java:111)
org.apache.hadoop.hdds.utils.db.DBStoreBuilder.build(DBStoreBuilder.java:227)
org.apache.hadoop.ozone.om.SnapshotDefragService.processCheckpointDb(SnapshotDefragService.java:226)
org.apache.hadoop.ozone.om.SnapshotDefragService.performFullDefragmentation(SnapshotDefragService.java:311)
org.apache.hadoop.ozone.om.SnapshotDefragService.triggerSnapshotDefragOnce(SnapshotDefragService.java:1026)
org.apache.hadoop.ozone.om.SnapshotDefragService$SnapshotDefragTask.call(SnapshotDefragService.java:942)
org.apache.hadoop.hdds.utils.BackgroundService$PeriodicalTask.lambda$run$0(BackgroundService.java:171)
java.base/java.util.concurrent.CompletableFuture$AsyncRun.run$$$capture(CompletableFuture.java:1736)
java.base/java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java)
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:829)

Note: The following is a snippet that can be used to trigger the leak before the fix:

    DBStoreBuilder checkpointDbBuilder = DBStoreBuilder.newBuilder(ozoneManager.getConfiguration())
        .setName(checkpointDirName)
        .setPath(Paths.get(defraggedDbPath).getParent())
        .setCreateCheckpointDirs(false);

    for (String cfName : COLUMN_FAMILIES_TO_TRACK_IN_SNAPSHOT) {
      checkpointDbBuilder.addTable(cfName);
    }

    try (RDBStore checkpointStore = checkpointDbBuilder.build()) {
      // ...
    }

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-13860

How was this patch tested?

  • Add one more test case that triggers this condition.

@smengcl smengcl requested a review from Copilot October 31, 2025 04:15
Copy link
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull Request Overview

This PR fixes a resource leak by properly closing TableConfig objects created for extra column families discovered in the database. The change ensures that column family options allocated for extra tables (those not in the provided families set) are properly released in both success and error paths.

  • Extracted extra variable declaration outside the try block to enable cleanup in the finally block
  • Added a finally block to close extra TableConfig objects after database opening

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Copy link
Contributor

@sadanand48 sadanand48 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @smengcl for the patch, LGTM

@smengcl smengcl merged commit 72167cf into apache:master Oct 31, 2025
90 of 91 checks passed
@smengcl smengcl deleted the HDDS-13860-fix-leak branch October 31, 2025 23:41
@smengcl
Copy link
Contributor Author

smengcl commented Oct 31, 2025

Thanks @sadanand48 for reviewing this.

Copy link
Contributor

@jojochuang jojochuang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably a bug from HDDS-6323 but it is used only by a test so it's okay.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants