Skip to content

Conversation

@swamirishi
Copy link
Contributor

@swamirishi swamirishi commented Nov 10, 2025

What changes were proposed in this pull request?

Currently bootstrap lock is acquired after the snapshot is already opened thus this can lead to a deadlock condition during bootstrap where the Bootstrap flow has already acquired a bootstrap lock and is waiting on snapshot cache lock to be acquired which cannot be acquired since the snapshots are still open. All background services acquire bootstrap lock while still having a snapshot open which is going to cause a deadlock here.
To fix this all background services should always acquire bootstrap lock before opening a snapshot. The only con to this is that the entire task of background service would be blocked when the bootstrap copy batch is running on the leader om which should be ok since bootstrap would be an infrequent operation.
However one possible improvement would be to just to create a hardlink(which we already do to ensure this file doesn't get deleted by rocksdb operations but we also write the file into the Tarball stream synchronously

public static long linkAndIncludeFile(File file, String entryName,
ArchiveOutputStream<TarArchiveEntry> archiveOutput, Path tmpDir) throws IOException {
File link = tmpDir.resolve(entryName).toFile();
long bytes = 0;
try {
Files.createLink(link.toPath(), file.toPath());
TarArchiveEntry entry = archiveOutput.createArchiveEntry(link, entryName);
archiveOutput.putArchiveEntry(entry);
try (InputStream input = Files.newInputStream(link.toPath())) {
bytes = IOUtils.copyLarge(input, archiveOutput);
}
archiveOutput.closeArchiveEntry();
} catch (IOException ioe) {
LOG.error("Couldn't create hardlink for file {} while including it in tarball.",
file.getAbsolutePath(), ioe);
throw ioe;
} finally {
Files.deleteIfExists(link.toPath());
}
return bytes;
}
) to the file to be copied into tar outputStream into a tmp directory under the bootstrap lock and write all the entries corresponding to the link created outside the lock(The entry should also include the hardLinkFile created in the last batch).
The above can be done as part of a separate patch @sadanand48
https://issues.apache.org/jira/browse/HDDS-13906

What is the link to the Apache JIRA

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

How was this patch tested?

Existing unit tests which was stuck because of the deadlock and marked flaky because of 272544aa95c66d0743f9ac94b2f4d586b325b8a7

…o deadlock

Change-Id: I0e8a540848cbaddc1e476ccb537dca18134251d1
@adoroszlai adoroszlai removed their request for review November 10, 2025 06:24
@swamirishi swamirishi added the snapshot https://issues.apache.org/jira/browse/HDDS-6517 label Nov 10, 2025
@swamirishi
Copy link
Contributor Author

@adoroszlai this is the fix for HDDS-13889

@swamirishi swamirishi removed the request for review from adoroszlai November 10, 2025 06:33
@adoroszlai
Copy link
Contributor

@adoroszlai this is the fix for HDDS-13889

Great, please run flaky-test-check for TestSnapshotBackgroundServices.

@adoroszlai
Copy link
Contributor

Error:  Tests run: 8, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 28.62 s <<< FAILURE! -- in org.apache.hadoop.ozone.om.service.TestKeyDeletingService$Normal
Error:  org.apache.hadoop.ozone.om.service.TestKeyDeletingService$Normal.testKeyDeletingServiceWithDeepCleanedSnapshots -- Time elapsed: 0.837 s <<< ERROR!
java.lang.ClassCastException: class org.apache.hadoop.hdds.utils.BackgroundTaskQueue cannot be cast to class org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService$DeletingServiceTaskQueue (org.apache.hadoop.hdds.utils.BackgroundTaskQueue and org.apache.hadoop.ozone.om.service.AbstractKeyDeletingService$DeletingServiceTaskQueue are in unnamed module of loader 'app')
	at org.apache.hadoop.ozone.om.service.KeyDeletingService.getTasks(KeyDeletingService.java:484)
	at org.apache.hadoop.ozone.om.service.KeyDeletingService.getTasks(KeyDeletingService.java:1)
	at java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:733)
	at org.apache.hadoop.hdds.utils.BackgroundService.runPeriodicalTaskNow(BackgroundService.java:107)
	at org.apache.hadoop.ozone.om.service.TestKeyDeletingService$Normal.testKeyDeletingServiceWithDeepCleanedSnapshots(TestKeyDeletingService.java:663)

Please wait for clean CI run in fork before opening PR.

@swamirishi swamirishi marked this pull request as draft November 10, 2025 13:32
Change-Id: I09e75187190855c430d4d5848d313f6f3ba87a66
@swamirishi swamirishi marked this pull request as ready for review November 10, 2025 15:09
@swamirishi
Copy link
Contributor Author

@adoroszlai this is the fix for HDDS-13889

Great, please run flaky-test-check for TestSnapshotBackgroundServices.

Ran it 10x10 times all of them have passed https://github.com/swamirishi/ozone/actions/runs/19223007061

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 deadlock issue where bootstrap lock acquisition was happening after snapshots were opened, causing background services to deadlock with the bootstrap flow. The fix ensures that all background services acquire the bootstrap lock before opening any snapshots, preventing the deadlock condition.

Key Changes

  • Replaced Semaphore-based locking with ReadWriteLock in BootstrapStateHandler.Lock to allow concurrent read access for background services while bootstrap takes an exclusive write lock
  • Introduced DeletingServiceTaskQueue in AbstractKeyDeletingService that wraps all background tasks to automatically acquire read locks before execution
  • Updated SstFilteringService to acquire bootstrap lock before opening snapshots instead of after

Reviewed Changes

Copilot reviewed 17 out of 17 changed files in this pull request and generated 4 comments.

Show a summary per file
File Description
BootstrapStateHandler.java Core locking mechanism changed from Semaphore to ReadWriteLock with separate read/write lock acquisition methods
AbstractKeyDeletingService.java Added DeletingServiceTaskQueue wrapper class that automatically acquires read lock before task execution
KeyDeletingService.java Updated to use DeletingServiceTaskQueue and removed bootstrap lock acquisition from individual operations
DirectoryDeletingService.java Updated to use DeletingServiceTaskQueue and removed manual bootstrap lock handling
SnapshotDeletingService.java Updated to use DeletingServiceTaskQueue and removed manual bootstrap lock handling
SstFilteringService.java Moved bootstrap lock acquisition to before snapshot opening instead of during SST file operations
RocksDBCheckpointDiffer.java Updated lock acquisition calls to use new acquireReadLock() API
DBCheckpointServlet.java Updated Lock implementation to return UncheckedAutoCloseable instead of self-reference
OMDBCheckpointServlet.java Updated Lock to aggregate multiple locks and return composite closeable
OMDBCheckpointServletInodeBasedXfer.java Updated to use new acquireWriteLock() API
Test files Updated test code to use new lock API and removed flaky test annotation from previously deadlocking test
Comments suppressed due to low confidence (3)

hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:1266

  • This catch block is unreachable because acquireReadLock() doesn't actually throw InterruptedException (the underlying lock.lock() method is not interruptible). Either switch to lockInterruptibly() in the base Lock implementation, or remove this catch block.
    } catch (InterruptedException e) {
      throw new RuntimeException(e);
    }

hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:1105

  • This catch block is unreachable because acquireReadLock() doesn't actually throw InterruptedException (the underlying lock.lock() method is not interruptible). Either switch to lockInterruptibly() in the base Lock implementation, or remove this catch block.
    } catch (InterruptedException e) {
      throw new RuntimeException(e);
    }

hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:664

  static class Lock extends BootstrapStateHandler.Lock {

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

class Lock implements AutoCloseable {
private final Semaphore semaphore = new Semaphore(1);
/** Bootstrap state handler lock implementation. Should be always acquired before opening any snapshot to avoid
* deadlocks*/
Copy link

Copilot AI Nov 10, 2025

Choose a reason for hiding this comment

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

Missing space after period in the comment. Should be "Should be always acquired before opening any snapshot to avoid deadlocks."

Suggested change
* deadlocks*/
* deadlocks */

Copilot uses AI. Check for mistakes.
* is wrapped such that its execution acquires a read lock via
* {@code getBootstrapStateLock().acquireReadLock()} before performing any
* operations. The lock is automatically released upon task completion or
* exception, ensuring safe concurrent execution of tasks within the service when running along with bootstrap flow.
Copy link

Copilot AI Nov 10, 2025

Choose a reason for hiding this comment

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

[nitpick] The Javadoc comment should end with "ensures safe concurrent execution of tasks within the service when running alongside bootstrap flow." (changed "along with" to "alongside" for better grammar).

Suggested change
* exception, ensuring safe concurrent execution of tasks within the service when running along with bootstrap flow.
* exception, ensuring safe concurrent execution of tasks within the service when running alongside bootstrap flow.

Copilot uses AI. Check for mistakes.
Comment on lines +39 to +44
public UncheckedAutoCloseable acquireWriteLock() throws InterruptedException {
return lock(false);
}

@Override
public void close() {
unlock();
public UncheckedAutoCloseable acquireReadLock() throws InterruptedException {
return lock(true);
Copy link

Copilot AI Nov 10, 2025

Choose a reason for hiding this comment

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

The methods acquireWriteLock() and acquireReadLock() declare throws InterruptedException, but they cannot actually throw this exception. The underlying lock.lock() call (line 35) is a blocking operation that doesn't throw InterruptedException.

If you need interruptible locking, use lock.lockInterruptibly() instead. Otherwise, remove the throws InterruptedException declaration from both methods.

Copilot uses AI. Check for mistakes.
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.

My understanding of the change, is that previously, the bootstrap lock is held when submitting internal requests (directory delete, key delete). That is moved to much later and bootstrap lock is held when the requests are being executed.


// Submit Purge paths request to OM. Acquire bootstrap lock when processing deletes for snapshots.
try (BootstrapStateHandler.Lock lock = snapTableKey != null ? getBootstrapStateLock().lock() : null) {
try {
Copy link
Contributor

Choose a reason for hiding this comment

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

lock removed here.

.setPurgeKeysRequest(requestBuilder.build()).setClientId(getClientId().toString()).build();

try (Lock lock = snapTableKey != null ? getBootstrapStateLock().lock() : null) {
try {
Copy link
Contributor

Choose a reason for hiding this comment

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

lock removed here.

.setClientId(clientId.toString())
.build();

try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

lock removed here.

.setSnapshotMoveTableKeysRequest(moveDeletedKeys)
.setClientId(clientId.toString())
.build();
try (BootstrapStateHandler.Lock lock = getBootstrapStateLock().lock()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

lock removed here.


try (
UncheckedAutoCloseableSupplier<OmSnapshot> snapshotMetadataReader =
// Acquire bootstrap lock before opening any snapshot.
Copy link
Contributor

Choose a reason for hiding this comment

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

moved the bootstrap lock earlier

Comment on lines +35 to +36
lock.lock();
return lock::unlock;
Copy link
Contributor

Choose a reason for hiding this comment

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

This line returns a method reference to unlock the acquired lock. Specifically, it creates an UncheckedAutoCloseable instance that, when closed, calls unlock() on the same lock, releasing it. This enables usage of the lock in a try-with-resources style, ensuring the lock will be released when no longer needed.

Change-Id: Ib5a872948e090cbf1f3cc51dd3a92f159429e835

# Conflicts:
#	hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SstFilteringService.java
Change-Id: Icc3420a005fe68db59aeb9ede68a7f6cc4e22831
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.

LGTM

@jojochuang jojochuang merged commit 20471a7 into apache:master Nov 12, 2025
43 checks passed
@jojochuang
Copy link
Contributor

Merged. Thanks @swamirishi

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

Labels

snapshot https://issues.apache.org/jira/browse/HDDS-6517

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants