[ 
https://issues.apache.org/jira/browse/HDDS-13860?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Siyao Meng updated HDDS-13860:
------------------------------
    Description: 
(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:

https://github.com/apache/ozone/blob/f30870f2a229a2a618f3199aa047d3397e77a9cd/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableConfig.java#L58-L60

{code:title=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
{code}

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

{code}
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)
...
{code}

{code}
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)
{code}

  was:
(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:

https://github.com/apache/ozone/blob/f30870f2a229a2a618f3199aa047d3397e77a9cd/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableConfig.java#L58-L60

{code:title=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
{code}

But those options aren't properly released later, leading to leaks:

{code}

{code}


> RocksDatabase#open leaks extra column families when failing to read cfOptions 
> from disk
> ---------------------------------------------------------------------------------------
>
>                 Key: HDDS-13860
>                 URL: https://issues.apache.org/jira/browse/HDDS-13860
>             Project: Apache Ozone
>          Issue Type: Bug
>            Reporter: Siyao Meng
>            Assignee: Siyao Meng
>            Priority: Major
>
> (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:
> https://github.com/apache/ozone/blob/f30870f2a229a2a618f3199aa047d3397e77a9cd/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/TableConfig.java#L58-L60
> {code:title=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
> {code}
> But those options aren't properly released later, leading to leaks, e.g. 
> output:
> {code}
> 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)
> ...
> {code}
> {code}
> 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)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to