kfaraz opened a new pull request, #13369:
URL: https://github.com/apache/druid/pull/13369

   ### Description
   
   In a cluster with a large number of streaming tasks (~1000), 
`SegmentAllocateAction`s on the overlord can often take very long intervals of 
time to finish thus causing spikes in the `task/action/run/time`. This may 
result in lag building up while a task waits for a segment to get allocated.
   
   The root causes are:
   - large number of metadata calls made to the segments and pending segments 
tables
   - `giant` lock held in `TaskLockbox.tryLock()` to acquire task locks and 
allocate segments
   
   Since the contention typically arises when several tasks of the same 
datasource try to allocate segments for the same interval/granularity, the 
allocation run times can be improved by batching the requests together.
   
   ### Changes
   
   #### Broad strokes
   - Queue handling: `SegmentAllocationQueue`
   - Rest of the behaviour remains exactly the same (order of steps, retries, 
etc.)
   - Redundant metadata calls have been eliminated
   - Query to fetch matching pending segment has been made into a single query
   - `giant` lock is acquired just once per batch per tryInterval in 
`TaskLockbox`
   - Old code has been retained for reference and in case we want to put this 
behind a feature flag
   
   #### Adding a request
   - Add method `SegmentAllocateAction.performAsync()` which returns a 
`Future<SegmentId>`
   - Submit each allocate action to a `SegmentAllocationQueue`
   - Add the action to a batch and execute it when it is due
   
   #### Processing a request batch
   - Wait a certain amount of time (currently 5s) before processing a batch
   - Send all requests in the batch to `TaskLockbox` to acquire task locks and 
allocate segments
   - Respond to leadership changes and stop processing queue when not leader
   - Emit batch and request level metrics
   
   #### Allocating segments
   - Add `TaskLockbox.allocateSegments()` retaining the logic from 
`TaskLockbox.tryLock()`
   - Add `IndexerMetadataStorageCoordinator.allocatePendingSegments()` 
retaining the logic from `allocatePendingSegment()`
   
   #### New metrics
   Per batch (dims: `dataSource`, ):
   - `task/action/batch/runTime`
   - `task/action/batch/queueTime`
   - `task/action/batch/size`
   
   Per request (dims: `taskId`, `taskType`, `dataSource`):
   - `task/action/attempt/count`
   - `task/action/success/count`
   
   ### Pending changes
   - Tests for new flow
   - Docs for metrics
   - (Optional) feature flag for new flow
   
   #### Release note
   Speed up segment allocation and reduce metadata calls by clubbing multiple 
requests together.
   
   <hr>
   
   This PR has:
   
   - [ ] been self-reviewed.
      - [ ] using the [concurrency 
checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md)
 (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked 
related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in 
[licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code 
wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, 
ensuring the threshold for [code 
coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md)
 is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


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