Skip to content

Fix parallel segment reload race on IndexLoadingConfig tier; add IndexLoadingConfig.copy() to avoid per-segment ZK fetches#18174

Open
rsrkpatwari1234 wants to merge 11 commits intoapache:masterfrom
rsrkpatwari1234:rsrkpatwari1234-issue-18164
Open

Fix parallel segment reload race on IndexLoadingConfig tier; add IndexLoadingConfig.copy() to avoid per-segment ZK fetches#18174
rsrkpatwari1234 wants to merge 11 commits intoapache:masterfrom
rsrkpatwari1234:rsrkpatwari1234-issue-18164

Conversation

@rsrkpatwari1234
Copy link
Copy Markdown
Contributor

@rsrkpatwari1234 rsrkpatwari1234 commented Apr 12, 2026

Problem

When multiple segments were reloaded in parallel (reloadAllSegments / batched reloadSegments), all tasks shared a single IndexLoadingConfig from one fetchIndexLoadingConfig() call. Each reload path calls setSegmentTier(...) (and related updates) on that shared instance, so concurrent tasks could overwrite each other’s tier. With tier overrides in table config, that could apply the wrong preprocessing / loading settings (#18164).

Fix

  • BaseTableDataManager.reloadSegments: Renamed to reloadSegmentDataManagersInParallel and calls fetchIndexLoadingConfig() once per batch, then for each parallel task pass indexLoadingConfigTemplate.copy() into reloadSegment, so every segment gets its own config for tier and other per-segment mutation.
  • IndexLoadingConfig.copy(): New method that builds a new IndexLoadingConfig with the same instance / table / schema references and tableDataDir, without copying segmentTier (each copy starts clean, like a fresh fetch). This keeps correctness while avoiding N repeated ZK reads (one fetch + N light copies instead of N fetches).

Tests

IndexLoadingConfigTest: Asserts copy shares TableConfig / Schema, matches tableDataDir, does not inherit segmentTier from the template, and that tier changes on the copy do not affect the original.

Fixes #18164

@codecov-commenter
Copy link
Copy Markdown

codecov-commenter commented Apr 12, 2026

Codecov Report

✅ All modified and coverable lines are covered by tests.
✅ Project coverage is 63.23%. Comparing base (7e10a36) to head (f8e2537).

Additional details and impacted files
@@             Coverage Diff              @@
##             master   #18174      +/-   ##
============================================
+ Coverage     63.18%   63.23%   +0.05%     
  Complexity     1616     1616              
============================================
  Files          3214     3214              
  Lines        195838   195842       +4     
  Branches      30251    30251              
============================================
+ Hits         123734   123836     +102     
+ Misses        62236    62105     -131     
- Partials       9868     9901      +33     
Flag Coverage Δ
custom-integration1 100.00% <ø> (ø)
integration 100.00% <ø> (ø)
integration1 100.00% <ø> (ø)
integration2 0.00% <ø> (ø)
java-11 63.19% <100.00%> (+0.07%) ⬆️
java-21 63.13% <100.00%> (-0.03%) ⬇️
temurin 63.23% <100.00%> (+0.05%) ⬆️
unittests 63.22% <100.00%> (+0.05%) ⬆️
unittests1 55.43% <100.00%> (+0.04%) ⬆️
unittests2 34.82% <42.85%> (+0.04%) ⬆️

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

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

@rsrkpatwari1234
Copy link
Copy Markdown
Contributor Author

Requesting review on this. Integration test failure seems unrelated to this PR -

java.lang.AssertionError: [ExactlyOnce] Transaction markers were not propagated within 120s; committed records are not visible to read_committed consumers. read_committed=0, read_uncommitted=153636
	at org.apache.pinot.integration.tests.ExactlyOnceKafkaRealtimeClusterIntegrationTest.waitForCommittedRecordsVisible(ExactlyOnceKafkaRealtimeClusterIntegrationTest.java:181)

}

private void reloadSegments(List<SegmentDataManager> segmentDataManagers, IndexLoadingConfig indexLoadingConfig,
private void reloadSegmentDataManagersInParallel(List<SegmentDataManager> segmentDataManagers,
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

I think we can keep the original name, and mention the parallel to this method's javadoc if you feel like.

IndexLoadingConfig copy = new IndexLoadingConfig(_instanceDataManagerConfig, _tableConfig, _schema);
copy.setTableDataDir(_tableDataDir);
return copy;
}
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Let's not have this method. This doesn't copy everything in this object (e.g. _readMode is not copied here), which would fail to meet the semantic of "copy". Unless you make sure everything is honored while performing the copy.

_segmentReloadSemaphore.acquire(segmentName, _logger);
try {
reloadSegment(segmentDataManager, indexLoadingConfig, forceDownload);
reloadSegment(segmentDataManager, indexLoadingConfigTemplate.copy(), forceDownload);
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

If we have 100k of segments then we'll have as many copies of IndexLoadingConfig objects here, with only the segment tier aren't the same.

Segment tiers are expected to be only a few per server. Can we have, for example, a map from tier to index loading config so we only need to create as many copies as the amount of tiers?

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.

Potential Race Condition against Segment Tier during Reload All Segments

3 participants