Invalidate cache load status on historical as soon as it is served#14616
Invalidate cache load status on historical as soon as it is served#14616kfaraz wants to merge 17 commits intoapache:masterfrom
Conversation
| public void resolve() | ||
| { | ||
| synchronized (statusRefs) { | ||
| synchronized (requestStatusesLock) { |
There was a problem hiding this comment.
I am trying to understand what the implication is of switching from statusRefs to requestStatusesLock is. However, this file doesn't use @GuardedBy and there isn't a comment on requestStatusesLock explaining how it's supposed to be used. Would you be able to add one or both of those, to make it easier to understand the synchronization logic?
There was a problem hiding this comment.
Thanks for calling it out. Yes, it is better if I just add a comment explaining the logic.
I made the switch to make sure that the update invoked by requestStatuses.invalidate() later in this method is safeguarded.
The existing synchronization on statusRefs essentially meant that only one thread can be executing the resolve of a given CustomSettableFuture at any point. That condition still holds but with less concurrency, i.e. in the new code, only one thread can be executing the resolve of any CustomSettableFuture at any point.
So, in other words, while multiple segments might be loaded by the historical in parallel, the futures will only be resolved serially.
This will not have any other impact on the correctness or performance of this code.
There was a problem hiding this comment.
Thanks for the explanation. It looks like there is no longer anything synchronizing on statusRefs, so the new code is somewhat simpler. Please do add a comment to requestStatusesLock, or use @GuardedBy on the things it is meant to guard, as appropriate, in order to make the logic easier to follow for others.
There was a problem hiding this comment.
Added an @GuardedBy for requestStatuses and some comments.
| ); | ||
| final List<DataSegmentChangeRequestAndStatus> result = new ArrayList<>(statusRefs.size()); | ||
| statusRefs.forEach((request, statusRef) -> { | ||
| // Remove complete statuses from the cache |
There was a problem hiding this comment.
When is this resolve method called exactly? Is it possible that these invalidations are too late, i.e. is it still possible that an even faster load->drop->load would be racey?
Put another way: would it make sense to invalidate the cache on receipt of the drop, rather than here?
There was a problem hiding this comment.
As long as the coordinator doesn't give up on a request it has sent to the historical, we should be good.
i.e. once the coordinator has asked a historical to LOAD a segment, it should not ask the historical to DROP the same segment until the LOAD has failed or succeeded.
Otherwise, we might keep the cached value of LOAD success around and serve it to the second load request which comes after the drop.
There was a problem hiding this comment.
This code could probably do with some cleanup and doing what had been originally suggested in this comment i.e. maintaining the cache (or really just a concurrent map) at the DataSegment level itself rather than DataSegmentChangeRequest.
But I wanted to keep the footprint of this PR low so that we could include it in Druid 27 release. We can take up a more holistic overhaul later. Let me know what you think.
There was a problem hiding this comment.
once the coordinator has asked a historical to LOAD a segment, it should not ask the historical to DROP the same segment until the LOAD has failed or succeeded.
Could this happen after Coordinator leader change or restart?
There was a problem hiding this comment.
Yes, I feel those are the only cases when this is likely to happen.
|
Also moved out class |
|
@gianm , there is another case which the current code was behaving weirdly. If a load keeps failing repeatedly on a historical, the coordinator may always get a PENDING response rather than FAILED. Typical scenario:
|
|
Trying to handle all these case, I have made the following changes:
@gianm , let me know what you think. |
| if (segmentsToDrop.remove(segment)) { | ||
| segmentManager.dropSegment(segment); | ||
|
|
||
| File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getId().toString()); |
Check failure
Code scanning / CodeQL
Uncontrolled data used in path expression
|
There is one more case that requires handling, a concurrent load and drop of the same segment. I am currently working on this, will update the PR soon. |
When would this occur? |
Only if the client (Coordinator) sends a LOAD request and then sends a DROP request for the same segment while the load is still in progress. There is already a list of Proposed fixBetter handling of the
Typical scenario
Similarly a DROP that comes before a LOAD might result in partial or complete delete of segment files but the LOAD task might continue to think that it succeeded. |
|
@kfaraz Thanks for outlining the scenario. IMO, it makes sense to include this fix, since it seems straightforward. |
gianm
left a comment
There was a problem hiding this comment.
Most of the changes LGTM. I had a couple of questions.
| loaded = segmentManager.loadSegment( | ||
| segment, | ||
| lazy, | ||
| () -> unannounceAndDropSegment(segment, DataSegmentChangeCallback.NOOP), |
There was a problem hiding this comment.
I take it all these changes from removeSegment to unannounceAndDropSegment are so we do an immediate drop (rather than delayed drop) in the case of failed loads? If so— that seems right to me. No reason to delay the drop if we never loaded and announced it in the first place.
There was a problem hiding this comment.
Yes, that is the intention here. But just to clarify, this was already the case and has not been introduced by this PR. I have only renamed the methods to remove ambiguity.
| * from the cache.</li> | ||
| * </ol> | ||
| * <p> | ||
| * Maximum size of this cache must be significantly greater than the number of |
There was a problem hiding this comment.
What happens if it's not significantly greater?
I guess this could happen if someone sets druid.coordinator.loadqueuepeon.http.batchSize on the Coordinator higher than druid.segmentCache.statusQueueMaxSize on the Historical.
There was a problem hiding this comment.
If the Coordinator sends a batch larger than the cache size, the Historical would not be able to have an entry for some of the requests in the cache. These requests would still be processed but their finished statuses would never be updated in the cache and thus never sent back to the Coordinator.
The Coordinator would end up retrying these requests and the Historical might duplicate some work but there would never be any correctness issue, since we are not sending any response to the Coordinator for the un-cached requests.
Given that the default cache size is 100 and default request batch size is 1, we should be okay. We can update the docs to mention that the cache size must be at least 10x of the batch size.
There was a problem hiding this comment.
Interestingly, druid.segmentCache.statusQueueMaxSize is not documented yet.
There was a problem hiding this comment.
Where does the 10x factor come from?
There was a problem hiding this comment.
Where does the 10x factor come from?
Just a safe-side measure in case Coordinator sends multiple batches of different requests in quick succession, such that the total number of unique requests exceeds the cache size. Not really based on any concrete calculations. 😅
There was a problem hiding this comment.
Some more thoughts on this:
- The
Cachecan really just be aMap. The only advantage theCachemight offer is auto-expiry of requests that the Coordinator gave up on, but I don't think that is currently being used in the code anyway. - Maybe we shouldn't even have a config for
batchSize. The Coordinator would start by sending a single request to the historicals and in each HTTP response, each Historical could indicate how many requests it would like to take in the next batch. This could be based on the number of loading threads, number of pending requests. - Given that the number of pending requests is always going to be small, we should just get rid of the
cacheSizeconfig and hard-code it to something like 1000. And if in any case, the Coordinator sends more requests than a Historical can handle, it immediately fails the request with an appropriate error message.
What do you think?
There was a problem hiding this comment.
I would vote for removing the cache structure - as discussed internally as well, if we are relying on HTTP status codes for correctness I don't think relying on a cache structure on server side is fully reliable.
Another easy solution when a defined amount of queued request is reached could be to return some retrying HTTP response to the call so that if needed, the load queue peon can retry that request after sometime.
There was a problem hiding this comment.
The
Cachecan really just be aMap. The only advantage theCachemight offer is auto-expiry of requests that the Coordinator gave up on, but I don't think that is currently being used in the code anyway.
The Coordinator could crash and never follow up on a request, so it's still useful to expire stuff.
Maybe we shouldn't even have a config for batchSize. The Coordinator would start by sending a single request to the historicals and in each HTTP response, each Historical could indicate how many requests it would like to take in the next batch. This could be based on the number of loading threads, number of pending requests.
Seems like a good idea! In addition, in case the Coordinator sends more commands than the Historical asks for, the Historical could reject the extra commands. (Although I am not sure if the protocol allows this.)
Given that the number of pending requests is always going to be small, we should just get rid of the cacheSize config and hard-code it to something like 1000.
Sounds good to me. Fewer configs are good.
Btw, I am wondering now, what are the scenarios where the Historical might receive multiple copies of the same load or drop command? Is this something that happens in typical operation, or in some kind of abnormal case?
There was a problem hiding this comment.
Btw, I am wondering now, what are the scenarios where the Historical might receive multiple copies of the same load or drop command?
Do you mean the same action on same segment or different actions on same segment? Both of these can actually happen under normal conditions:
Same action on same segment
Even under normal operation, the Coordinator keeps sending the same request to a Historical until the request finishes.
Essentially, the Coordinator sends a batch of requests to the Historical and waits for a response. Historical sends back a response as soon as any one of the requests in this batch (or even a previously submitted batch which is still being tracked) finishes. So chances are that the Coordinator gets back a response with one request having status SUCCESS or FAILED and the others PENDING, because the Historical was still working on them when it completed the HTTP request. The Coordinator then re-sends the requests for which it had received a PENDING status. By then, the Historical might have already finished them and cached their results.
Different actions on same segment but after a delay
The case for a different action (i.e. load after drop) without ever having read the status of the previous action can happen during coordinator restarts or leader re-elections. Although, in these cases, it is not likely that the drop and load would happen concurrently.
Different actions on same segment happening concurrently
This is not expected to happen under normal conditions, and can only be caused by the Coordinator being in some weird state or a bug in the code.
There was a problem hiding this comment.
Even under normal operation, the Coordinator keeps sending the same request to a Historical until the request finishes.
Ah, thanks for explaining. I didn't realize it was that common.
|
A couple of doubts :
Are we sure that the DROP request actually gets stuck in pending state? It appears from the current code (https://github.com/apache/druid/blob/master/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java#L463) that the status is marked success after either the delete request is scheduled async or run inline.
|
Yes, @rohangarg , thanks for pointing this out. I have updated my comment. The DROP request is marked a success immediately after unannouncing the segment. Repeated back and forth in the concurrency cases of this code got me turned around a bit 😅 . Even though this particular case is probably not that big of an issue (marking a segment as drop success even though the files are still there), it might lead to the coordinator assigning historicals beyond their disk capacity. I have tried to handle such cases now though and they should work as expected. Please let me know if you have any further thoughts.
The response payload has not changed, only the Java class name has changed. So the changes are backward compatible. Please let me know if you notice a discrepancy in this. |
|
Removed from 27.0 in favor of #14670. |
|
Closing this as this implementation is still buggy and the design requires some further thought. |
Description
Follow up to #11717
This PR attempts to fix the behaviour in the following scenarios:
Case 1: Coordinator gets incorrect response in case of a quick load-drop-load
Case 2: Coordinator sees a repeatedly failing load as PENDING rather than FAILED
druid/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java
Lines 562 to 563 in efb3281
Case 3:
segmentsToDropsegmentsToDropand proceeds with downloading the segment.segmentsToDropanymore.Similarly a drop that comes before a load might result in partial or complete delete of segment files but the load task might continue to think that it succeeded.
Classes to review
SegmentLoadDropHandlerSegmentLoadDropHandlerTestChanges
DataSegmentChangeRequestAndStatustoDataSegmentChangeResponseDataSegmentChangeResponseout ofSegmentLoadDropHandleras it is a POJO returned to the client and need not be tied to a specific loader implementation.requestStatusescache to be a map fromDataSegmentto the fullDataSegmentChangeResponseRelease note
Fix bug on historical when a load-drop-load of a segment happens in quick succession.
This PR has: