gianm commented on code in PR #14616: URL: https://github.com/apache/druid/pull/14616#discussion_r1273291195
########## server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java: ########## @@ -90,19 +93,46 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final SegmentManager segmentManager; private final ScheduledExecutorService exec; private final ServerTypeConfig serverTypeConfig; - private final ConcurrentSkipListSet<DataSegment> segmentsToDelete; + private final ConcurrentSkipListSet<DataSegment> segmentsToDrop; private final SegmentCacheManager segmentCacheManager; private volatile boolean started = false; - // Keep history of load/drop request status in a LRU cache to maintain idempotency if same request shows up - // again and to return status of a completed request. Maximum size of this cache must be significantly greater - // than number of pending load/drop requests. so that history is not lost too quickly. - private final Cache<DataSegmentChangeRequest, AtomicReference<Status>> requestStatuses; + /** + * Used to cache the status of a completed load or drop request until it has + * been served to the (Coordinator) client exactly once. + * <p> + * The cache is used as follows: + * <ol> + * <li>An entry with state PENDING is added to the cache upon receiving a + * request to load or drop a segment.</li> + * <li>A duplicate request received at this point is immediately answered with PENDING.</li> + * <li>Once the load/drop finishes, the entry is updated to either SUCCESS or FAILED.</li> + * <li>A duplicate request received at this point is immediately answered with + * SUCCESS or FAILED and the entry is removed from the cache.</li> + * <li>If the first request itself finishes after the load or drop has already + * completed, it is answered with a SUCCESS or FAILED and the entry is removed + * from the cache.</li> + * </ol> + * <p> + * Maximum size of this cache must be significantly greater than the number of Review Comment: > The `Cache` can really just be a `Map`. The only advantage the `Cache` might 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? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
