resume partially offline replicas in controller validation#17754
resume partially offline replicas in controller validation#17754rohityadav1993 wants to merge 1 commit intoapache:masterfrom
Conversation
Codecov Report❌ Patch coverage is
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
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
|
Pinot Tests / Pinot Integration Test Set 1 - Unrelated failure in |
|
cc: @tarun11Mavani, @Jackie-Jiang for review. |
|
| } | ||
| } else if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS | ||
| && _isPartialOfflineReplicaRepairEnabled) { | ||
| // Handle case where some replicas are OFFLINE while others are CONSUMING |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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 |
| updateInstanceStatesForNewConsumingSegment(instanceStatesMap, latestSegmentName, null, segmentAssignment, | ||
| instancePartitionsMap); | ||
| } | ||
| } else if (latestSegmentZKMetadata.getStatus() == Status.IN_PROGRESS |
There was a problem hiding this comment.
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.
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
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.
The periodicity of RealtimeSegmentValidationManager is configrable, default: 3600 so this wouldn't be too aggressive if configured correctly. |
noob-se7en
left a comment
There was a problem hiding this comment.
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.
|
I dont think this is a bugfix. |
Offline -> Consuming errors are currently handled by updating IS to offline(there is a intermediate step when EV would go in error state but IS is always updated to Offline state, RealtimeSegmentDataManager.java#L1418). The error states in EV happens when the segment is supposet to be stopped after commit criteria. But we are only checking segment metadata and IS to take decision here.
For rebalance scenario for consuming segments the RealtimeSegmentAssigner does not update consuming segments to OFFLINE state but directly to CONSUMING state(RealtimeSegmentAssignment.java#L245-L257), so there is no scenario where we have a consuming segment set to OFFLINE state which we are trying to update in this PR. We also have both these operations happening only in Leader controller for the table and behind optmisitic lock for IS update to avoid race conditions. |
|
Only for below code in offline -> consuming we mark segment as offline right? I believe for any other exceptions like occuring from failures in creation of stream message decoder, etc causes the segment to go in a error state. I will recheck about the rebalance case, Thanks for providing additional context. I have thought about implementing this solution before but I was thinking of coming up with some solution where we could just remove the dependency of calling controller in an infinte loop from server until controller marks the segment in offline state. I couldn't find a way to mark segment in error state from inside consumer thread if it runs into some exception. If we can do that then controller just resets the error segments. |
|
@noob-se7en , The way the class has been designed is, EV will always go to ERROR state for the segment except if the segment is IN_PROGRESS in propertyStore(metadata) where the controller puts the segment in OFFLINE state in IS after getting signaled from RSDM when postStopConsumedMsg invoked. Please do review this, for safety I have put this behind a controller config and tested in a docker set up. |
There was a problem hiding this comment.
Pull request overview
This PR adds an opt-in controller-side validation/repair path to recover from realtime LLC segments that end up with mixed replica states (some CONSUMING, some OFFLINE) for IN_PROGRESS segments—allowing the controller to flip OFFLINE replicas back to CONSUMING so they can retry consumption (addressing #11314).
Changes:
- Added a new controller config flag
controller.realtime.segment.partialOfflineReplicaRepairEnabled(defaultfalse). - Updated
PinotLLCRealtimeSegmentManager.ensureAllPartitionsConsuming(...)to detectIN_PROGRESSsegments withOFFLINEreplicas and reset them toCONSUMINGwhen the flag is enabled. - Added unit tests covering both enabled and disabled behavior.
Reviewed changes
Copilot reviewed 3 out of 3 changed files in this pull request and generated no comments.
| File | Description |
|---|---|
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java |
Implements the mixed CONSUMING/OFFLINE detection and repair for IN_PROGRESS segments when enabled. |
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerConf.java |
Introduces the new configuration key and accessor (isPartialOfflineReplicaRepairEnabled()). |
pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManagerTest.java |
Adds tests verifying OFFLINE→CONSUMING repair when enabled and no-op when disabled. |
|
Can I get a merge for this if it looks good. @chenboat, @Jackie-Jiang, @xiangfu0 |
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
controller.realtime.segment.partialOfflineReplicaRepairEnabled(defaults tofalse)PinotLLCRealtimeSegmentManagervalidation to detect and repair mixed CONSUMING/OFFLINE replica statesImplementation Details
Configuration:
controller.realtime.segment.partialOfflineReplicaRepairEnabledinControllerConffalse(opt-in for backward compatibility)Repair Logic:
Testing:
Unit Tests:
Local cluster test pLan:
echo "nameserver 0.0.0.0" > /etc/resolv.confin server-1Upgrade Notes
This feature is disabled by default. To enable, set:
controller.realtime.segment.partialOfflineReplicaRepairEnabled=true