kfaraz commented on code in PR #14616:
URL: https://github.com/apache/druid/pull/14616#discussion_r1273126224


##########
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:
   > Where does the 10x factor come from?
   
   Just a safe-side measure in case Coordinator sends multiple batches of 
different requests. Not really based on any concrete thoughts.



##########
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:
   Some more thoughts on this:
   - 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.
   - 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 `cacheSize` config 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?



-- 
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]

Reply via email to