pulsar icon indicating copy to clipboard operation
pulsar copied to clipboard

[fix][broker] fix prepareInitPoliciesCacheAsync in SystemTopicBasedTopicPoliciesService

Open TakaHiR07 opened this issue 1 month ago • 8 comments

Fixes #24977

Motivation

As shown in the issue, fix two problem: 1. cleanCacheAndCloseReader() executed twice cause concurrent error, which result in too many orphan reader remain in SystemTopicBasedTopicPoliciesService 2. double update in policyCacheInitMap cause recursive update error

Modifications

  1. do cleanPoliciesCacheInitMap only once when throw exception
  2. avoid double update in policyCacheInitMap. use putIfAbsent instead of computeIfAbsent. It is not appropriate to add so many operation into compute().
  3. add two test, to simulate if throw exception in createReader, initPolicyCache, readMorePolicy of prepareInitPoliciesCacheAsync. By the way, it seems lack of unittest in SystemTopicBasedTopicPoliciesService.
  4. "newReader()" remove some logic, it is confused when readCompletableFuture throw exception.
  5. not remove cleanPoliciesCacheInitMap() in initPolicesCache() when closed.get()==true, since broker is closed, clean twice is ok.

There is one point should be consider in this pr

  1. When use putIfAbsent, if too many getTopicPolicy() trigger prepareInitPoliciesCacheAsync, it would generate many empty completableFuture. Further more, we can use double check in the code to avoid this object gc.(the code would be ugly).

Besides, this case still exist: if failed to close reader in cleanCacheAndCloseReader(), this closing reader maybe have chance to reconnect and become orphan reader. This is not this pr's work.

Verifying this change

  • [ ] Make sure that the change passes the CI checks.

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • [ ] Dependencies (add or upgrade a dependency)
  • [ ] The public API
  • [ ] The schema
  • [ ] The default values of configurations
  • [ ] The threading model
  • [ ] The binary protocol
  • [ ] The REST endpoints
  • [ ] The admin CLI options
  • [ ] The metrics
  • [ ] Anything that affects deployment

Documentation

  • [ ] doc
  • [ ] doc-required
  • [x] doc-not-needed
  • [ ] doc-complete

TakaHiR07 avatar Nov 14 '25 09:11 TakaHiR07

Codecov Report

:white_check_mark: All modified and coverable lines are covered by tests. :white_check_mark: Project coverage is 74.29%. Comparing base (6fdb4b9) to head (33ae945). :warning: Report is 10 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@            Coverage Diff            @@
##             master   #24980   +/-   ##
=========================================
  Coverage     74.29%   74.29%           
- Complexity    34026    34066   +40     
=========================================
  Files          1920     1920           
  Lines        150252   150252           
  Branches      17428    17428           
=========================================
+ Hits         111634   111636    +2     
- Misses        29706    29735   +29     
+ Partials       8912     8881   -31     
Flag Coverage Δ
inttests 26.17% <75.00%> (-0.39%) :arrow_down:
systests 22.87% <67.85%> (-0.02%) :arrow_down:
unittests 73.84% <100.00%> (+0.02%) :arrow_up:

Flags with carried forward coverage won't be shown. Click here to find out more.

Files with missing lines Coverage Δ
.../service/SystemTopicBasedTopicPoliciesService.java 77.86% <100.00%> (+0.19%) :arrow_up:

... and 78 files with indirect coverage changes

:rocket: New features to boost your workflow:
  • :snowflake: Test Analytics: Detect flaky tests, report on failures, and find test suite problems.
  • :package: JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

codecov-commenter avatar Nov 14 '25 11:11 codecov-commenter

Which logic could cause this issue ?

Request-1: policyCacheInitMap put future1
Request-1: create reader1
Request-1: readerCaches put reader1
reader1 read error
Request-1: first time cleanCacheAndCloseReader(), include:
        remove reader1 in readerCaches
        close reader1
        remove future1 in policyCacheInitMap

Request-2: policyCacheInitMap put future2
Request-1: second time cleanCacheAndCloseReader(), only remove future2 in policyCacheInitMap
Request-2: create reader2
Request-2: readerCaches put reader2

Request-3: policyCacheInitMap put future3
Request-3: create reader3
Request-3: readerCaches put reader3

Technoboy- avatar Nov 19 '25 04:11 Technoboy-

Which logic could cause this issue ?

Request-1: policyCacheInitMap put future1
Request-1: create reader1
Request-1: readerCaches put reader1
reader1 read error
Request-1: first time cleanCacheAndCloseReader(), include:
        remove reader1 in readerCaches
        close reader1
        remove future1 in policyCacheInitMap

Request-2: policyCacheInitMap put future2
Request-1: second time cleanCacheAndCloseReader(), only remove future2 in policyCacheInitMap
Request-2: create reader2
Request-2: readerCaches put reader2

Request-3: policyCacheInitMap put future3
Request-3: create reader3
Request-3: readerCaches put reader3

Is this bug existed in the 3.0.x , not the latest version ?

Technoboy- avatar Nov 19 '25 04:11 Technoboy-

Which logic could cause this issue ?

@Technoboy- restart broker with version-3.0.x. Restart broker-1, and after a few time restart broker-2. When load topic and getTopicPolicy on broker-1, the corresponding __change_event topic on broker-2 is unload.

I don't use the latest version. Maybe in latest version, this concurrent case is avoid by pr-24658. But it still catch the exception and cleanCacheAndPolicyMap twice, this is dangerous.

TakaHiR07 avatar Nov 19 '25 04:11 TakaHiR07

Which logic could cause this issue ?

@Technoboy- restart broker with version-3.0.x. Restart broker-1, and after a few time restart broker-2. When load topic and getTopicPolicy on broker-1, the corresponding __change_event topic on broker-2 is unload.

I don't use the latest version. Maybe in latest version, this concurrent case is avoid by pr-24658. But it still catch the exception and cleanCacheAndPolicyMap twice, this is dangerous.

How could the latest code cause the issue ? I'm not understand

Technoboy- avatar Nov 19 '25 04:11 Technoboy-

How could the latest code cause the issue ? I'm not understand

You can see the code in branch-3.0. Latest code is a bit different, the concurrent case is found on branch-3.0

TakaHiR07 avatar Nov 19 '25 06:11 TakaHiR07

How could the latest code cause the issue ? I'm not understand

You can see the code in branch-3.0. Latest code is a bit different, the concurrent case is found on branch-3.0

So it's better to fix it to branch-3.0, for the master branch, I don't think it's needed.

Technoboy- avatar Nov 19 '25 08:11 Technoboy-

So it's better to fix it to branch-3.0, for the master branch, I don't think it's needed.

@Technoboy- I think it is better to also fix in master branch. Since the current code in master branch is for improvement, not a true fix, and still have risk.

TakaHiR07 avatar Nov 19 '25 09:11 TakaHiR07

As shown in the issue, fix two problem: 1. cleanCacheAndCloseReader() executed twice cause concurrent error, which result in too many orphan reader remain in SystemTopicBasedTopicPoliciesService 2. double update in policyCacheInitMap cause recursive update error

I think that this problem exists also in master branch and therefore merging this PR and cherry-picking it to maintenance branches makes sense.

lhotari avatar Dec 10 '25 14:12 lhotari

Depends on #24658 for branch-4.0 and branch-4.1

lhotari avatar Dec 15 '25 08:12 lhotari

Flaky test #25081, please take a look

lhotari avatar Dec 16 '25 10:12 lhotari