Skip to content

Conversation

@swamirishi
Copy link
Contributor

@swamirishi swamirishi commented Jun 22, 2025

What changes were proposed in this pull request?

Currently even though the java doc of the base class BackgroundService says it would wait for the entire run to finish before triggering the next run, it doesn't wait as it just submits the tasks to the queue and doesn't track if those tasks have finished.
Added a wait at the end of CompletableFuture to fix it.

What is the link to the Apache JIRA

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

How was this patch tested?

No unit tests added unit test for the same.

…efore triggering next run

Change-Id: I2569ad4a0ab27e900c4f6d3cae98b0e4735503ca
Change-Id: I5f4e826ca73cc43d5e81b4df7289382bf31cf510
}, 100, 3000);
Thread.sleep(3000);
assertEquals(values, IntStream.range(0, 10).boxed().map(map::get).collect(Collectors.toList()));
lockList.forEach(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.

minor nit :
check that the even indices are also processed after unlock and maybe check for size of values (5)before and after unlock (10)

Copy link
Contributor

@adoroszlai adoroszlai left a comment

Choose a reason for hiding this comment

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

Thanks @swamirishi for the patch.

Comment on lines 170 to 172
} catch (Throwable e) {
LOG.error("Background service execution failed", e);
} finally {
Copy link
Contributor

Choose a reason for hiding this comment

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

Re-throw Error.

} finally {
long endTime = System.nanoTime();
if (endTime - serviceStartTime > serviceTimeoutInNanos) {
LOG.warn("{} Background service execution failed which took {}ns > {}ns(timeout)",
Copy link
Contributor

Choose a reason for hiding this comment

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

Are you sure we get here only if execution failed?

}
return true;
}, 100, 3000);
Thread.sleep(3000);
Copy link
Contributor

Choose a reason for hiding this comment

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

Please don't use fixed sleep.

Comment on lines 94 to 95
lockList.forEach(Lock::unlock);
backgroundService.shutdown();
Copy link
Contributor

Choose a reason for hiding this comment

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

Release/shutdown in finally.

Comment on lines 83 to 88
if (i % 2 == 1 && map.get(i) != 1) {
return false;
}
if (i % 2 == 0 && map.get(i) != 0) {
return false;
}
Copy link
Contributor

Choose a reason for hiding this comment

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

i % 2 and map.get(i) can be compared directly.

Comment on lines 47 to 62
Map<Integer, Integer> map = new HashMap<>();
Map<Integer, Lock> locks = new HashMap<>();
for (int i = 0; i < 10; i++) {
int index = i;
locks.put(index, new ReentrantLock());
map.put(index, 0);
queue.add(() -> {
locks.get(index).lock();
try {
map.compute(index, (k, v) -> v == null ? 1 : (v + 1));
} finally {
locks.get(index).unlock();
}
return new BackgroundTaskResult.EmptyTaskResult();
});
}
Copy link
Contributor

Choose a reason for hiding this comment

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

To simplify the test case, please extract an inner class for the task, which should keep track of its invocations, and support locking.

@SaketaChalamchala
Copy link
Contributor

Thanks for the review @sadanand48 and @adoroszlai. Could you review the updated patch?

@SaketaChalamchala
Copy link
Contributor

@jojochuang and @smengcl could you also review this patch?

@adoroszlai
Copy link
Contributor

Please do take a look at CI results in fork.

M M IS: Inconsistent synchronization of org.apache.hadoop.hdds.utils.BackgroundService.exec; locked 83% of time  Unsynchronized access at BackgroundService.java:[line 189]
M D IM: Check for oddness that won't work for negative numbers in org.apache.hadoop.util.TestBackgroundService.lambda$null$3(int)  At TestBackgroundService.java:[line 131]

https://github.com/swamirishi/ozone/actions/runs/16352154299/job/46201433990

hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java
 23: Unused import - java.util.concurrent.CompletionException.

https://github.com/swamirishi/ozone/actions/runs/16352154299/job/46201434007#step:16:17

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 marked this pull request as ready for review July 24, 2025 15:17
… finish before launching next set and exiting.
@SaketaChalamchala
Copy link
Contributor

Updated the patch. Instead of the runner waiting for all the tasks to complete at the end,

  • Runner waits at the beginning for the previous set of tasks to complete
  • Runner Submits the next set of tasks to the thread pool and exits.
    This way, even with a single thread the worker tasks get to complete because runner just submits the task and exits and the next scheduled run will not start until all the previous tasks are complete.
    cc @swamirishi
    @jojochuang , @adoroszlai @smengcl @sadanand48 could you please review the update?

@jojochuang
Copy link
Contributor

jojochuang commented Jul 25, 2025

findbugs: M M IS: Inconsistent synchronization of org.apache.hadoop.hdds.utils.BackgroundService.exec; locked 75% of time Unsynchronized access at BackgroundService.java:[line 92]

@SaketaChalamchala please run ./hadoop-ozone/dev-support/checks/findbugs.sh locally to confirm the next fix. Thanks!

saketa and others added 3 commits July 25, 2025 18:48
SaketaChalamchala pushed a commit to SaketaChalamchala/ozone that referenced this pull request Jul 26, 2025
Copy link
Contributor Author

@swamirishi swamirishi left a comment

Choose a reason for hiding this comment

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

LGTM excepting for a few nitpicky comments.


public abstract BackgroundTaskQueue getTasks();

protected void execTaskCompletion() { }
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't think this is required

Copy link
Contributor

Choose a reason for hiding this comment

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

I added this so that it might be useful to perform post task completion steps like updating last run metrics or incrementing run count or processing the result of the background tasks if necessary

} catch (RuntimeException e) {
LOG.error("Background service execution failed.", e);
} finally {
execTaskCompletion();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

We can simply addAndIncreament while submitting

@smengcl smengcl added the snapshot https://issues.apache.org/jira/browse/HDDS-6517 label Jul 28, 2025
@jojochuang
Copy link
Contributor

Looks good to me. I'll merge it to unblock #8655

@jojochuang jojochuang merged commit ce5676a into apache:master Jul 30, 2025
42 checks passed
@jojochuang
Copy link
Contributor

@adoroszlai
Copy link
Contributor

Author:     Swaminathan Balachandran <swamirishi.sb@gmail.com>
AuthorDate: Wed Jul 30 13:49:54 2025 -0400
Commit:     GitHub <noreply@github.com>
CommitDate: Wed Jul 30 10:49:54 2025 -0700

    HDDS-13320. BackgroundService should wait for the completion of run before triggering next run (#8677)
    
    Co-authored-by: Wei-Chiu Chuang <weichiu@apache.org>

When merging, please make sure to credit contributors. @SaketaChalamchala had 7 of 10 commits in this PR.

SaketaChalamchala pushed a commit to SaketaChalamchala/ozone that referenced this pull request Jul 30, 2025
jojochuang added a commit to jojochuang/ozone that referenced this pull request Jul 31, 2025
…efore triggering next run (apache#8677)

Co-authored-by: Wei-Chiu Chuang <weichiu@apache.org>
Comment on lines -191 to +200
List<? extends DatanodeDetails> dnList = cluster().getStorageContainerManager()
.getScmNodeManager()
.getAllNodes();
NodeManager nodeManager = cluster().getStorageContainerManager().getScmNodeManager();
List<? extends DatanodeDetails> dnList = nodeManager.getAllNodes();

for (DatanodeDetails dn : dnList) {
final int expected = cluster().getStorageContainerManager().getScmNodeManager().getContainers(dn).size();

List<HddsProtos.DatanodeUsageInfoProto> usageInfoList =
storageClient.getDatanodeUsageInfo(
dn.getIpAddress(), dn.getUuidString());

assertEquals(1, usageInfoList.size());
assertEquals(expected, usageInfoList.get(0).getContainerCount());
assertEquals(nodeManager.getContainers(dn).size(), usageInfoList.get(0).getContainerCount());
Copy link
Contributor

Choose a reason for hiding this comment

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

This change in TestContainerOperations seems completely unrelated to the rest of the PR.

Tejaskriya pushed a commit to Tejaskriya/ozone that referenced this pull request Jul 31, 2025
…efore triggering next run (apache#8677)

Co-authored-by: Wei-Chiu Chuang <weichiu@apache.org>
swamirishi added a commit to swamirishi/ozone that referenced this pull request Dec 3, 2025
…ion of run before triggering next run (apache#8677)

Co-authored-by: Wei-Chiu Chuang <weichiu@apache.org>
(cherry picked from commit ce5676a)

 Conflicts:
	hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java
	hadoop-hdds/common/src/main/java/org/apache/hadoop/util/TestBackgroundService.java
	hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestContainerOperations.java

Change-Id: Ie41fb7a56755364a31aefe6d8936a5849bf03b42
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.

6 participants