Skip to content

HDDS-14800. Guard RocksDB iterator against closed DB during volume failure#9904

Open
priyeshkaratha wants to merge 8 commits intoapache:masterfrom
priyeshkaratha:HDDS-14800
Open

HDDS-14800. Guard RocksDB iterator against closed DB during volume failure#9904
priyeshkaratha wants to merge 8 commits intoapache:masterfrom
priyeshkaratha:HDDS-14800

Conversation

@priyeshkaratha
Copy link
Contributor

What changes were proposed in this pull request?

When StorageVolumeChecker detects a volume failure and calls failVolume(), it closes the underlying RocksDB instance while BackgroundContainerDataScanner or OnDemandContainerScanner may still hold an active iterator over that DB, calling native RocksDB methods on a closed DB can cause a crash. This fix adds two complementary guards to RDBStoreAbstractIterator: a fast-fail check so hasNext() returns false immediately once the DB is closed (stopping the scan loop without touching native code), and reference counting by acquiring a slot on RocksDatabase.counter at iterator creation and releasing it on close(), so the existing waitAndClose() mechanism waits for all iterators to finish before physically closing the DB. Together these ensure the scan exits cleanly and the DB cannot be destroyed while an iterator is still in use.

What is the link to the Apache JIRA

HDDS-14800

How was this patch tested?

Tested using added unit test cases.

@ChenSammi
Copy link
Contributor

@priyeshkaratha , could you add a unit test for BackgroundContainerDataScanner?

@priyeshkaratha priyeshkaratha marked this pull request as ready for review March 11, 2026 10:39
@Gargi-jais11
Copy link
Contributor

Gargi-jais11 commented Mar 13, 2026

Let's add some test for the race condition scenarios.

Copy link
Contributor

@devmadhuu devmadhuu left a comment

Choose a reason for hiding this comment

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

@priyeshkaratha overall changes LGTM +1. Just few nits. Pls check.


@Override
public final boolean hasNext() {
if (isDbClosed()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

The isClosed() fast-fail only protects hasNext(). Methods like seekToFirst(), seekToLast(), seek() call native RocksDB directly without any check.

}
}

private void waitAndClose() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Please put a comment - "// Wait until all active operations (including open iterators) complete.
// Iterators acquired after DB close is triggered will fast-fail in
// hasNext(), so this loop is expected to complete quickly in practice."

public void close() {
rocksDBIterator.close();
if (dbRef != null) {
dbRef.close();
Copy link
Contributor

Choose a reason for hiding this comment

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

dbRef is final — it is never nulled out. If close() is called twice, dbRef.close() runs twice. Since dbRef holds counter::getAndDecrement, the counter gets decremented twice. Java's Closeable contract requires that calling close() more than once is safe.

rocksDBIterator.close();
if (dbRef != null) {
dbRef.close();
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
}
private final AtomicBoolean iteratorClosed = new AtomicBoolean(false);
@Override
public void close() {
if (iteratorClosed.compareAndSet(false, true)) {
rocksDBIterator.close();
if (dbRef != null) {
dbRef.close();
}
}
}

} catch (RocksDatabaseException e) {
LOG.warn("Failed to acquire DB reference for iterator on table {}: {}",
table.getName(), e.getMessage());
return null;
Copy link
Contributor

Choose a reason for hiding this comment

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

Actually here exception swallowed, , no reference counting guard ? Am I missing something here ?

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.

5 participants