Skip to content

resume partially offline replicas in controller validation#17754

Open
rohityadav1993 wants to merge 1 commit intoapache:masterfrom
rohityadav1993:11314-rohity
Open

resume partially offline replicas in controller validation#17754
rohityadav1993 wants to merge 1 commit intoapache:masterfrom
rohityadav1993:11314-rohity

Conversation

@rohityadav1993
Copy link
Contributor

@rohityadav1993 rohityadav1993 commented Feb 24, 2026

Description

This change introduces automated repair for partially offline replicas in realtime segment consumption for both pauseless and non-pausless Realtime tables. This addresses scenarios(issue: #11314) where some replicas fail during initialization (e.g., KafkaConsumer errors) and mark themselves OFFLINE while other replicas continue consuming normally.

Changes

  • Added new configuration flag controller.realtime.segment.partialOfflineReplicaRepairEnabled (defaults to false)
  • Enhanced PinotLLCRealtimeSegmentManager validation to detect and repair mixed CONSUMING/OFFLINE replica states
  • When enabled, controller automatically resets OFFLINE replicas back to CONSUMING state for IN_PROGRESS segments, allowing retry

Implementation Details

Configuration:

  • New property: controller.realtime.segment.partialOfflineReplicaRepairEnabled in ControllerConf
  • Default: false (opt-in for backward compatibility)

Repair Logic:

  • Detects segments with mixed CONSUMING/OFFLINE replica states during validation
  • Logs repair actions with details (segment name, offline count, instance list)
  • Resets identified OFFLINE replicas to CONSUMING state

Testing:
Unit Tests:

  • Added unit tests for enabled scenario (verifies OFFLINE→CONSUMING transition)
  • Added unit tests for disabled scenario (verifies no-op behavior)

Local cluster test pLan:

  • Set up realtime table(kafka) with two servers and two replicas, partialOfflineReplicaRepairEnabled = true
  • Mangle DNS config: echo "nameserver 0.0.0.0" > /etc/resolv.conf in server-1
  • Force commit, new consuming server on server-1 comes up in error state and moves to OFFLINE state while server-2 is in CONSUMING state
  • Run controller validation job: RealtimeSegmentValidationManager
  • The replica becomes healthy in CONSUMING state.

Upgrade Notes

This feature is disabled by default. To enable, set:
controller.realtime.segment.partialOfflineReplicaRepairEnabled=true

@codecov-commenter
Copy link

codecov-commenter commented Feb 24, 2026

Codecov Report

❌ Patch coverage is 93.33333% with 1 line in your changes missing coverage. Please review.
✅ Project coverage is 63.21%. Comparing base (8845675) to head (bd8b0f5).
⚠️ Report is 6 commits behind head on master.

Files with missing lines Patch % Lines
.../core/realtime/PinotLLCRealtimeSegmentManager.java 92.85% 0 Missing and 1 partial ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #17754      +/-   ##
============================================
- Coverage     63.23%   63.21%   -0.02%     
- Complexity     1448     1454       +6     
============================================
  Files          3176     3176              
  Lines        191025   191041      +16     
  Branches      29206    29211       +5     
============================================
- Hits         120788   120775      -13     
- Misses        60814    60838      +24     
- Partials       9423     9428       +5     
Flag Coverage Δ
custom-integration1 100.00% <ø> (?)
integration 100.00% <ø> (+100.00%) ⬆️
integration1 100.00% <ø> (?)
integration2 0.00% <ø> (ø)
java-11 63.15% <93.33%> (-0.02%) ⬇️
java-21 63.19% <93.33%> (+0.01%) ⬆️
temurin 63.21% <93.33%> (-0.02%) ⬇️
unittests 63.21% <93.33%> (-0.02%) ⬇️
unittests1 55.55% <ø> (-0.04%) ⬇️
unittests2 34.10% <93.33%> (+0.01%) ⬆️

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.

@rohityadav1993
Copy link
Contributor Author

Pinot Tests / Pinot Integration Test Set 1 - Unrelated failure in BaseBrokerRoutingManager

@rohityadav1993 rohityadav1993 marked this pull request as ready for review February 24, 2026 06:22
@rohityadav1993
Copy link
Contributor Author

cc: @tarun11Mavani, @Jackie-Jiang for review.

@anuragrai16
Copy link
Contributor

  1. Can you add details on what is the current behaviour ? I believe it retries upto 5 times to create the consumer and once failed, it sends a segmentStoppedConsuming back to controller. Why cant increasing this retries or make it perpetual also not solve this issue ?

  2. If the underlying issue persists, we have a continuous loop of :
    Validation → Repair (OFFLINE→CONSUMING) → Helix triggers transition →
    Server tries to consume → Fails → segmentStoppedConsuming → OFFLINE →
    Next Validation → Repair again. Im wondering what sort of race conditions can be caused by this ? And whether the server trying to start the consumption perpetually on its own and publishing a metric of retries is a safer fix ?

  3. To prevent (2) happening frequently, do we wanna add a lastRepairTime - and send a new event only after x seconds/minutes ?

}
} else if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS
&& _isPartialOfflineReplicaRepairEnabled) {
// Handle case where some replicas are OFFLINE while others are CONSUMING
Copy link
Contributor

@anuragrai16 anuragrai16 Feb 24, 2026

Choose a reason for hiding this comment

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

Similar to other repairs, you should add max segment completion time check,

if (!isExceededMaxSegmentCompletionTime(realtimeTableName, latestSegmentName, currentTimeMs)) {
        continue; 
    }

This ensures your repair does not mess up any actively ongoing commit.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

Copy link
Contributor Author

@rohityadav1993 rohityadav1993 Feb 25, 2026

Choose a reason for hiding this comment

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

It is a harmless addition but I think this is not needed in this scenario. Segments are supposed to be IN_PROGRESS in metadata and we are not creating new consuming segment with a different metadata and neither we are making any change to mark segment DONE in IdealState

}
} else if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS
&& _isPartialOfflineReplicaRepairEnabled) {
// Handle case where some replicas are OFFLINE while others are CONSUMING
Copy link
Contributor

Choose a reason for hiding this comment

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

+1

updateInstanceStatesForNewConsumingSegment(instanceStatesMap, latestSegmentName, null, segmentAssignment,
instancePartitionsMap);
}
} else if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS
Copy link
Contributor

Choose a reason for hiding this comment

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

Since we are only checking for IN_PROGRESS status, this should work for pauseless consumption as well. Can you call this out in the PR description for clarity.

@rohityadav1993
Copy link
Contributor Author

  1. Can you add details on what is the current behaviour ? I believe it retries upto 5 times to create the consumer and once failed, it sends a segmentStoppedConsuming back to controller. Why cant increasing this retries or make it perpetual also not solve this issue ?

This is right, 5 retries with 2 seconds delay, so in 10 seconds we exhaust all attempts to recover. This happens in RealtimeSegmentDataManager.java#L1793-L1805
We can only have limited retries here to avoid indefiinitely waiting for any permanent underlying errors. Controller validation job is post these retries exhaust.

  1. If the underlying issue persists, we have a continuous loop of :
    Validation → Repair (OFFLINE→CONSUMING) → Helix triggers transition →
    Server tries to consume → Fails → segmentStoppedConsuming → OFFLINE →
    Next Validation → Repair again. Im wondering what sort of race conditions can be caused by this ? And whether the
    server trying to start the consumption perpetually on its own and publishing a metric of retries is a safer fix ?

This should be a fairly safe scenario as we are not committing/creating new segment with new metadata but basically resurrecting an offline segment to consuming only if segement is in IN_PROGRESS state in ZK metadata.

  1. To prevent (2) happening frequently, do we wanna add a lastRepairTime - and send a new event only after x seconds/minutes ?

The periodicity of RealtimeSegmentValidationManager is configrable, default: 3600 so this wouldn't be too aggressive if configured correctly.

@shauryachats shauryachats added feature bugfix release-notes Referenced by PRs that need attention when compiling the next release notes labels Feb 25, 2026
Copy link
Contributor

@noob-se7en noob-se7en left a comment

Choose a reason for hiding this comment

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

If offline -> consuming transition fails due to any error, The segment will be marked in ERROR state and the RVM job will auto-reset the segment.

If consumer thread fails then yes the IS of that segment is marked offline - but I wouldn't recommend changing the IS to consuming in RVM. The IS of the segment can also be offline due to a rebalance. Best solution will be to somehow mark segment in error state from consumer thread itself.

@noob-se7en
Copy link
Contributor

I dont think this is a bugfix.

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

Labels

bugfix feature release-notes Referenced by PRs that need attention when compiling the next release notes

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants