Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/queue/queue.go

360 lines
13 KiB

package queue
import (
"context"
"fmt"
"sync"
"time"
"github.com/grafana/dskit/services"
"github.com/pkg/errors"
"go.uber.org/atomic"
)
const (
// How frequently to check for disconnected queriers that should be forgotten.
forgetCheckPeriod = 5 * time.Second
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
anyQueue = ""
)
var (
ErrTooManyRequests = errors.New("too many outstanding requests")
ErrStopped = errors.New("queue is stopped")
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
ErrQueueWasRemoved = errors.New("the queue has been removed or moved to another position")
)
// QueueIndex is opaque type that allows to resume iteration over tenants between successive calls
// of RequestQueue.GetNextRequestForQuerier method.
type QueueIndex int // nolint:revive
Scheduler: Add query fairness control across multiple actors within a tenant (#8752) **What this PR does / why we need it**: This PR wires up the scheduler with the hierarchical queues. It is the last PR to implement https://github.com/grafana/loki/pull/8585. When these changes are in place, the client performing query requests can control their QoS (query fairness) using the `X-Actor-Path` HTTP header. This header controls in which sub-queue of the tenant's scheduler queue the query request is enqueued. The place within the hierarchy where it is enqueued defines the probability with which the request gets dequeued. A common use-case for this QoS control is giving each Grafana user within a tenant their fair share of query execution time. Any documentation is still missing and will be provided by follow-up PRs. **Special notes for your reviewer**: ```console $ gotest -count=1 -v ./pkg/scheduler/queue/... -test.run=TestQueryFairness === RUN TestQueryFairness === RUN TestQueryFairness/use_hierarchical_queues_=_false dequeue_qos_test.go:109: duration actor a 2.007765568s dequeue_qos_test.go:109: duration actor b 2.209088331s dequeue_qos_test.go:112: total duration 2.209280772s === RUN TestQueryFairness/use_hierarchical_queues_=_true dequeue_qos_test.go:109: duration actor b 605.283144ms dequeue_qos_test.go:109: duration actor a 2.270931324s dequeue_qos_test.go:112: total duration 2.271108551s --- PASS: TestQueryFairness (4.48s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_false (2.21s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_true (2.27s) PASS ok github.com/grafana/loki/pkg/scheduler/queue 4.491s ``` ```console $ gotest -count=5 -v ./pkg/scheduler/queue/... -bench=Benchmark -test.run=^$ -benchtime=10000x -benchmem goos: linux goarch: amd64 pkg: github.com/grafana/loki/pkg/scheduler/queue cpu: 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz BenchmarkGetNextRequest BenchmarkGetNextRequest/without_sub-queues BenchmarkGetNextRequest/without_sub-queues-8 10000 29337 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21348 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21595 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21189 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21602 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33770 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33596 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 34432 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33760 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33664 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 71405 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 59472 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 117163 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 106505 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 64374 ns/op 3200 B/op 300 allocs/op BenchmarkQueueRequest BenchmarkQueueRequest-8 10000 168391 ns/op 320588 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 166203 ns/op 320587 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 149518 ns/op 320584 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 219776 ns/op 320583 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 185198 ns/op 320597 B/op 1156 allocs/op PASS ok github.com/grafana/loki/pkg/scheduler/queue 64.648s ``` Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
2 years ago
// StartIndexWithLocalQueue is the index of the queue that starts iteration over local and sub queues.
var StartIndexWithLocalQueue QueueIndex = -2
// StartIndex is the index of the queue that starts iteration over sub queues.
var StartIndex QueueIndex = -1
// Modify index to start iteration on the same tenant, for which last queue was returned.
func (ui QueueIndex) ReuseLastIndex() QueueIndex {
Scheduler: Add query fairness control across multiple actors within a tenant (#8752) **What this PR does / why we need it**: This PR wires up the scheduler with the hierarchical queues. It is the last PR to implement https://github.com/grafana/loki/pull/8585. When these changes are in place, the client performing query requests can control their QoS (query fairness) using the `X-Actor-Path` HTTP header. This header controls in which sub-queue of the tenant's scheduler queue the query request is enqueued. The place within the hierarchy where it is enqueued defines the probability with which the request gets dequeued. A common use-case for this QoS control is giving each Grafana user within a tenant their fair share of query execution time. Any documentation is still missing and will be provided by follow-up PRs. **Special notes for your reviewer**: ```console $ gotest -count=1 -v ./pkg/scheduler/queue/... -test.run=TestQueryFairness === RUN TestQueryFairness === RUN TestQueryFairness/use_hierarchical_queues_=_false dequeue_qos_test.go:109: duration actor a 2.007765568s dequeue_qos_test.go:109: duration actor b 2.209088331s dequeue_qos_test.go:112: total duration 2.209280772s === RUN TestQueryFairness/use_hierarchical_queues_=_true dequeue_qos_test.go:109: duration actor b 605.283144ms dequeue_qos_test.go:109: duration actor a 2.270931324s dequeue_qos_test.go:112: total duration 2.271108551s --- PASS: TestQueryFairness (4.48s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_false (2.21s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_true (2.27s) PASS ok github.com/grafana/loki/pkg/scheduler/queue 4.491s ``` ```console $ gotest -count=5 -v ./pkg/scheduler/queue/... -bench=Benchmark -test.run=^$ -benchtime=10000x -benchmem goos: linux goarch: amd64 pkg: github.com/grafana/loki/pkg/scheduler/queue cpu: 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz BenchmarkGetNextRequest BenchmarkGetNextRequest/without_sub-queues BenchmarkGetNextRequest/without_sub-queues-8 10000 29337 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21348 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21595 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21189 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21602 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33770 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33596 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 34432 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33760 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33664 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 71405 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 59472 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 117163 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 106505 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 64374 ns/op 3200 B/op 300 allocs/op BenchmarkQueueRequest BenchmarkQueueRequest-8 10000 168391 ns/op 320588 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 166203 ns/op 320587 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 149518 ns/op 320584 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 219776 ns/op 320583 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 185198 ns/op 320597 B/op 1156 allocs/op PASS ok github.com/grafana/loki/pkg/scheduler/queue 64.648s ``` Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
2 years ago
if ui < StartIndex {
return ui
}
return ui - 1
}
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
type Limits interface {
// MaxConsumers returns the max consumers to use per tenant or 0 to allow all consumers to consume from the queue.
MaxConsumers(user string, allConsumers int) int
}
// Request stored into the queue.
type Request any
// RequestChannel is a channel that queues Requests
type RequestChannel chan Request
// RequestQueue holds incoming requests in per-tenant queues. It also assigns each tenant specified number of queriers,
// and when querier asks for next request to handle (using GetNextRequestForQuerier), it returns requests
// in a fair fashion.
type RequestQueue struct {
services.Service
connectedConsumers *atomic.Int32
mtx sync.Mutex
cond contextCond // Notified when request is enqueued or dequeued, or querier is disconnected.
queues *tenantQueues
stopped bool
metrics *Metrics
pool *SlicePool[Request]
}
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
func NewRequestQueue(maxOutstandingPerTenant int, forgetDelay time.Duration, limits Limits, metrics *Metrics) *RequestQueue {
q := &RequestQueue{
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
queues: newTenantQueues(maxOutstandingPerTenant, forgetDelay, limits),
connectedConsumers: atomic.NewInt32(0),
metrics: metrics,
pool: NewSlicePool[Request](1<<6, 1<<10, 2), // Buckets are [64, 128, 256, 512, 1024].
}
q.cond = contextCond{Cond: sync.NewCond(&q.mtx)}
q.Service = services.NewTimerService(forgetCheckPeriod, nil, q.forgetDisconnectedConsumers, q.stopping).WithName("request queue")
return q
}
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
// Enqueue puts the request into the queue.
// If request is successfully enqueued, successFn is called with the lock held, before any querier can receive the request.
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
func (q *RequestQueue) Enqueue(tenant string, path []string, req Request, successFn func()) error {
q.mtx.Lock()
defer q.mtx.Unlock()
if q.stopped {
return ErrStopped
}
config: adds `frontend.max-query-capacity` to tune per-tenant query capacity (#11284) **What this PR does / why we need it**: Adds a new config `frontend.max-query-capacity` that allows users to configure what portion of the the available querier replicas can be used by a tenant. `max_query_capacity` is the corresponding YAML option that can be configured in limits or runtime overrides. For example, setting this to 0.5 would allow a tenant to use half of the available queriers. This complements the existing `frontend.max-queriers-per-tenant`. When both are configured, the smaller value of the resulting querier replica count is considered: ``` min(frontend.max-queriers-per-tenant, ceil(querier_replicas * frontend.max-query-capacity)) ``` *All* queriers will handle requests for a tenant if neither limits are applied. **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: noticed that we don't pass down the shuffle sharding limits for frontend (only using it with schedulers) https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/loki/modules.go#L895 but the [docs](https://github.com/grafana/loki/blob/26f097162a856db48ecbd16bef2f0b750029855b/pkg/validation/limits.go#L276) mention that`frontend.max-queriers-per-tenant` applies to frontend as well. ``` This option only works with queriers connecting to the query-frontend / query-scheduler, not when using downstream URL. ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Co-authored-by: J Stickler <julie.stickler@grafana.com> Co-authored-by: Danny Kopping <danny.kopping@grafana.com>
2 years ago
queue, err := q.queues.getOrAddQueue(tenant, path)
if err != nil {
return fmt.Errorf("no queue found: %w", err)
}
// Optimistically increase queue counter for tenant instead of doing separate
// get and set operations, because _most_ of the time the increased value is
// smaller than the max queue length.
// We need to keep track of queue length separately because the size of the
// buffered channel is the same across all sub-queues which would allow
// enqueuing more items than there are allowed at tenant level.
queueLen := q.queues.perUserQueueLen.Inc(tenant)
if queueLen > q.queues.maxUserQueueSize {
q.metrics.discardedRequests.WithLabelValues(tenant).Inc()
// decrement, because we already optimistically increased the counter
q.queues.perUserQueueLen.Dec(tenant)
return ErrTooManyRequests
}
select {
Scheduler: Add query fairness control across multiple actors within a tenant (#8752) **What this PR does / why we need it**: This PR wires up the scheduler with the hierarchical queues. It is the last PR to implement https://github.com/grafana/loki/pull/8585. When these changes are in place, the client performing query requests can control their QoS (query fairness) using the `X-Actor-Path` HTTP header. This header controls in which sub-queue of the tenant's scheduler queue the query request is enqueued. The place within the hierarchy where it is enqueued defines the probability with which the request gets dequeued. A common use-case for this QoS control is giving each Grafana user within a tenant their fair share of query execution time. Any documentation is still missing and will be provided by follow-up PRs. **Special notes for your reviewer**: ```console $ gotest -count=1 -v ./pkg/scheduler/queue/... -test.run=TestQueryFairness === RUN TestQueryFairness === RUN TestQueryFairness/use_hierarchical_queues_=_false dequeue_qos_test.go:109: duration actor a 2.007765568s dequeue_qos_test.go:109: duration actor b 2.209088331s dequeue_qos_test.go:112: total duration 2.209280772s === RUN TestQueryFairness/use_hierarchical_queues_=_true dequeue_qos_test.go:109: duration actor b 605.283144ms dequeue_qos_test.go:109: duration actor a 2.270931324s dequeue_qos_test.go:112: total duration 2.271108551s --- PASS: TestQueryFairness (4.48s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_false (2.21s) --- PASS: TestQueryFairness/use_hierarchical_queues_=_true (2.27s) PASS ok github.com/grafana/loki/pkg/scheduler/queue 4.491s ``` ```console $ gotest -count=5 -v ./pkg/scheduler/queue/... -bench=Benchmark -test.run=^$ -benchtime=10000x -benchmem goos: linux goarch: amd64 pkg: github.com/grafana/loki/pkg/scheduler/queue cpu: 11th Gen Intel(R) Core(TM) i7-1185G7 @ 3.00GHz BenchmarkGetNextRequest BenchmarkGetNextRequest/without_sub-queues BenchmarkGetNextRequest/without_sub-queues-8 10000 29337 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21348 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21595 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21189 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/without_sub-queues-8 10000 21602 ns/op 1600 B/op 100 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33770 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33596 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 34432 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33760 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_1_level_of_sub-queues-8 10000 33664 ns/op 2400 B/op 200 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 71405 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 59472 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 117163 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 106505 ns/op 3200 B/op 300 allocs/op BenchmarkGetNextRequest/with_2_levels_of_sub-queues-8 10000 64374 ns/op 3200 B/op 300 allocs/op BenchmarkQueueRequest BenchmarkQueueRequest-8 10000 168391 ns/op 320588 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 166203 ns/op 320587 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 149518 ns/op 320584 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 219776 ns/op 320583 B/op 1156 allocs/op BenchmarkQueueRequest-8 10000 185198 ns/op 320597 B/op 1156 allocs/op PASS ok github.com/grafana/loki/pkg/scheduler/queue 64.648s ``` Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
2 years ago
case queue.Chan() <- req:
q.metrics.queueLength.WithLabelValues(tenant).Inc()
q.metrics.enqueueCount.WithLabelValues(tenant, fmt.Sprint(len(path))).Inc()
q.cond.Broadcast()
// Call this function while holding a lock. This guarantees that no querier can fetch the request before function returns.
if successFn != nil {
successFn()
}
return nil
default:
q.metrics.discardedRequests.WithLabelValues(tenant).Inc()
// decrement, because we already optimistically increased the counter
q.queues.perUserQueueLen.Dec(tenant)
return ErrTooManyRequests
}
}
// ReleaseRequests returns items back to the slice pool.
// Must only be called in combination with DequeueMany().
func (q *RequestQueue) ReleaseRequests(items []Request) {
q.pool.Put(items)
}
// DequeueMany consumes multiple items for a single tenant from the queue.
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// It blocks the execution until it dequeues at least 1 request and continue reading
// until it reaches `maxItems` requests or if no requests for this tenant are enqueued.
// The caller is responsible for returning the dequeued requests back to the
// pool by calling ReleaseRequests(items).
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
func (q *RequestQueue) DequeueMany(ctx context.Context, idx QueueIndex, consumerID string, maxItems int) ([]Request, QueueIndex, error) {
items := q.pool.Get(maxItems)
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
lastQueueName := anyQueue
for {
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
item, newIdx, newQueueName, isTenantQueueEmpty, err := q.dequeue(ctx, idx, lastQueueName, consumerID)
if err != nil {
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// the consumer must receive the items if tenants queue is removed,
// even if it has collected less than `maxItems` requests.
if errors.Is(err, ErrQueueWasRemoved) {
err = nil
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
return items, newIdx, err
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
lastQueueName = newQueueName
items = append(items, item)
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
idx = newIdx.ReuseLastIndex()
if len(items) == maxItems || isTenantQueueEmpty {
return items, newIdx, nil
}
}
}
// Dequeue find next tenant queue and takes the next request off of it. Will block if there are no requests.
// By passing tenant index from previous call of this method, querier guarantees that it iterates over all tenants fairly.
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// Even if the consumer used UserIndex.ReuseLastUser to fetch the request from the same tenant's queue, it does not provide
// any guaranties that the previously used queue is still at this position because another consumer could already read
// the last request and the queue could be removed and another queue is already placed at this position.
func (q *RequestQueue) Dequeue(ctx context.Context, last QueueIndex, consumerID string) (Request, QueueIndex, error) {
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
dequeue, queueIndex, _, _, err := q.dequeue(ctx, last, anyQueue, consumerID)
return dequeue, queueIndex, err
}
func (q *RequestQueue) dequeue(ctx context.Context, last QueueIndex, wantedQueueName string, consumerID string) (Request, QueueIndex, string, bool, error) {
q.mtx.Lock()
defer q.mtx.Unlock()
querierWait := false
FindQueue:
// We need to wait if there are no tenants, or no pending requests for given querier.
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// However, if `wantedQueueName` is not empty, the caller must not be blocked because it wants to read exactly from that queue, not others.
for (q.queues.hasNoTenantQueues() || querierWait) && ctx.Err() == nil && !q.stopped && wantedQueueName == anyQueue {
querierWait = false
q.cond.Wait(ctx)
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// If the current consumer wants to read from specific queue, but he does not have any queues available for him,
// return an error to notify that queue has been already removed.
if q.queues.hasNoTenantQueues() && wantedQueueName != anyQueue {
return nil, last, wantedQueueName, false, ErrQueueWasRemoved
}
if q.stopped {
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
return nil, last, wantedQueueName, false, ErrStopped
}
if err := ctx.Err(); err != nil {
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
return nil, last, wantedQueueName, false, err
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
queue, tenant, idx := q.queues.getNextQueueForConsumer(last, consumerID)
last = idx
if queue == nil {
// it can be a case the consumer has other tenants queues available for him,
// it allows the consumer to pass the wait block,
// but the queue with index `last+1` has been already removed,
// for example if another consumer has read the last request from that queue,
// and as long as this consumer wants to read from specific tenant queue,
// it's necessary to return `ErrQueueWasRemoved` error.
if wantedQueueName != anyQueue {
return nil, last, wantedQueueName, false, ErrQueueWasRemoved
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// otherwise, if wantedQueueName is empty, then this consumer will go to the wait block again
// and as long as `last` index is updated, next time the consumer will request the queue
// with the new index that was returned from `getNextQueueForConsumer`.
// There are no unexpired requests, so we can get back
// and wait for more requests.
querierWait = true
goto FindQueue
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
if wantedQueueName != anyQueue && wantedQueueName != queue.Name() {
// it means that the consumer received another tenants queue because it was already removed
// or another queue is already at this index
return nil, last, queue.Name(), false, ErrQueueWasRemoved
}
// Pick next request from the queue.
request := queue.Dequeue()
isTenantQueueEmpty := queue.Len() == 0
if isTenantQueueEmpty {
q.queues.deleteQueue(tenant)
}
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
q.queues.perUserQueueLen.Dec(tenant)
q.metrics.queueLength.WithLabelValues(tenant).Dec()
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
// Tell close() we've processed a request.
q.cond.Broadcast()
DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Vladyslav Diachenko <vlad.diachenko@grafana.com>
1 year ago
return request, last, queue.Name(), isTenantQueueEmpty, nil
}
func (q *RequestQueue) forgetDisconnectedConsumers(_ context.Context) error {
q.mtx.Lock()
defer q.mtx.Unlock()
if q.queues.forgetDisconnectedConsumers(time.Now()) > 0 {
// We need to notify goroutines cause having removed some queriers
// may have caused a resharding.
q.cond.Broadcast()
}
return nil
}
func (q *RequestQueue) stopping(_ error) error {
q.mtx.Lock()
defer q.mtx.Unlock()
for !q.queues.hasNoTenantQueues() && q.connectedConsumers.Load() > 0 {
q.cond.Wait(context.Background())
}
// Only stop after dispatching enqueued requests.
q.stopped = true
// If there are still goroutines in GetNextRequestForQuerier method, they get notified.
q.cond.Broadcast()
return nil
}
func (q *RequestQueue) RegisterConsumerConnection(querier string) {
q.connectedConsumers.Inc()
q.mtx.Lock()
defer q.mtx.Unlock()
q.queues.addConsumerToConnection(querier)
}
func (q *RequestQueue) UnregisterConsumerConnection(querier string) {
q.connectedConsumers.Dec()
q.mtx.Lock()
defer q.mtx.Unlock()
q.queues.removeConsumerConnection(querier, time.Now())
}
func (q *RequestQueue) NotifyConsumerShutdown(querierID string) {
q.mtx.Lock()
defer q.mtx.Unlock()
q.queues.notifyQuerierShutdown(querierID)
}
func (q *RequestQueue) GetConnectedConsumersMetric() float64 {
return float64(q.connectedConsumers.Load())
}
// contextCond is a *sync.Cond with Wait() method overridden to support context-based waiting.
type contextCond struct {
*sync.Cond
// testHookBeforeWaiting is called before calling Cond.Wait() if it's not nil.
// Yes, it's ugly, but the http package settled jurisprudence:
// https://github.com/golang/go/blob/6178d25fc0b28724b1b5aec2b1b74fc06d9294c7/src/net/http/client.go#L596-L601
testHookBeforeWaiting func()
}
// Wait does c.cond.Wait() but will also return if the context provided is done.
// All the documentation of sync.Cond.Wait() applies, but it's especially important to remember that the mutex of
// the cond should be held while Wait() is called (and mutex will be held once it returns)
func (c contextCond) Wait(ctx context.Context) {
// "condWait" goroutine does q.cond.Wait() and signals through condWait channel.
condWait := make(chan struct{})
go func() {
if c.testHookBeforeWaiting != nil {
c.testHookBeforeWaiting()
}
c.Cond.Wait()
close(condWait)
}()
// "waiting" goroutine: signals that the condWait goroutine has started waiting.
// Notice that a closed waiting channel implies that the goroutine above has started waiting
// (because it has unlocked the mutex), but the other way is not true:
// - condWait it may have unlocked and is waiting, but someone else locked the mutex faster than us:
// in this case that caller will eventually unlock, and we'll be able to enter here.
// - condWait called Wait(), unlocked, received a broadcast and locked again faster than we were able to lock here:
// in this case condWait channel will be closed, and this goroutine will be waiting until we unlock.
waiting := make(chan struct{})
go func() {
c.L.Lock()
close(waiting)
c.L.Unlock()
}()
select {
case <-condWait:
// We don't know whether the waiting goroutine is done or not, but we don't care:
// it will be done once nobody is fighting for the mutex anymore.
case <-ctx.Done():
// In order to avoid leaking the condWait goroutine, we can send a broadcast.
// Before sending the broadcast we need to make sure that condWait goroutine is already waiting (or has already waited).
select {
case <-condWait:
// No need to broadcast as q.cond.Wait() has returned already.
return
case <-waiting:
// q.cond.Wait() might be still waiting (or maybe not!), so we'll poke it just in case.
c.Broadcast()
}
// Make sure we are not waiting anymore, we need to do that before returning as the caller will need to unlock the mutex.
<-condWait
}
}