Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix cancel issue between Query Frontend and Query Schdeduler #5113

Merged
merged 12 commits into from
Jan 13, 2022
Merged

Fix cancel issue between Query Frontend and Query Schdeduler #5113

merged 12 commits into from
Jan 13, 2022

Conversation

kavirajk
Copy link
Contributor

@kavirajk kavirajk commented Jan 12, 2022

What this PR does / why we need it:
Query Frontend failed to send cancel signal even to Query Frontend Worker during some special cases(explained below)
And that makes querier to run till completion(even after cancellation) wasting lots of resources.

This scenario is analogous to this simple Go program (run in go-playground). The gist is default branch in go select loop takes always precedence when other branch operations are blocked (sending or receiving on the channel). In our case, we missed the sending of cancel signal

Now to the original issue.

The main observation is for every Querier that haven't received cancel, it's corresponding Scheduler also didn't get it.

  1. single query on QF has split into 128 sub queries
  2. And all of them got scheduled and picked up by the querier.
  3. I added some logs in two points. Where QF send sending cancel to scheduler, And scheduler loop sends it to querier.

Then among those 128 subqueries, only 8 of them got cancel from QF to Scheduler, remaining 120 got dropped in QF itself, never even reached the Scheduler.

The root cause is, whenever frontend worker loop is busy with service request on the main select branch ( lisitening on w.requestCh ), and if during that time, QF send cancel request to Scheduler via cancelCh it will get block for sometime on the other end (coz no buffer). During that time, it got into default case and thus ignoring the cancel signal even before sending it to Scheduler.

Basically we need to make sure to send cancel signal to Scheduler, no matter how busy the frontend worker to pickup cancel from the cancelCh.

Which issue(s) this PR fixes:
Fixes #5132

Special notes for your reviewer:

Checklist

  • Documentation added
  • Tests updated
  • Add an entry in the CHANGELOG.md about the changes.

@kavirajk kavirajk changed the title Fix cancel issue between Query Frontend -> Query Schdeduler Fix cancel issue between Query Frontend and Query Schdeduler Jan 12, 2022
Signed-off-by: Kaviraj <[email protected]>
Signed-off-by: Kaviraj <[email protected]>
@cyriltovena cyriltovena marked this pull request as ready for review January 13, 2022 09:47
@cyriltovena cyriltovena requested a review from a team as a code owner January 13, 2022 09:47
Copy link
Contributor

@cyriltovena cyriltovena left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@colega PTAL I want to make sure we didn't miss anything.

Signed-off-by: Kaviraj <[email protected]>
@colega
Copy link
Contributor

colega commented Jan 13, 2022

I agree that this fixes the issue but I think we're overcomplicating it a little bit. Cancelling the querier request isn't a strict need but just politeness and avoids wasting resources, so it would be great to ensure it, but not application-critical.

If there's no need to ensure that cancellation has happened, I think just adding some sensible capacity to each worker's cancelCh should be enough (while keeping the default: pass section, to avoid blocking if that cancellation channel is really full).

Let's say make(chan uint64, 1024): if we have more than 1024 canceled requests waiting for worker to be free, then we probably have bigger issues to worry about anyway.

WDYT?

Edit, I tried with this:

diff --git pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
index fa6395783..7d32f2f70 100644
--- pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
+++ pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
@@ -191,7 +191,7 @@ func newFrontendSchedulerWorker(conn *grpc.ClientConn, schedulerAddr string, fro
                schedulerAddr: schedulerAddr,
                frontendAddr:  frontendAddr,
                requestCh:     requestCh,
-               cancelCh:      make(chan uint64),
+               cancelCh:      make(chan uint64, 1024),
        }
        w.ctx, w.cancel = context.WithCancel(context.Background())

And the test from this PR also passes 👍

@cyriltovena
Copy link
Contributor

cyriltovena commented Jan 13, 2022

I agree that this fixes the issue but I think we're overcomplicating it a little bit. Cancelling the querier request isn't a strict need but just politeness and avoids wasting resources, so it would be great to ensure it, but not application-critical.

If there's no need to ensure that cancellation has happened, I think just adding some sensible capacity to each worker's cancelCh should be enough (while keeping the default: pass section, to avoid blocking if that cancellation channel is really full).

Let's say make(chan uint64, 1024): if we have more than 1024 canceled requests waiting for worker to be free, then we probably have bigger issues to worry about anyway.

WDYT?

Edit, I tried with this:

diff --git pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
index fa6395783..7d32f2f70 100644
--- pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
+++ pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
@@ -191,7 +191,7 @@ func newFrontendSchedulerWorker(conn *grpc.ClientConn, schedulerAddr string, fro
                schedulerAddr: schedulerAddr,
                frontendAddr:  frontendAddr,
                requestCh:     requestCh,
-               cancelCh:      make(chan uint64),
+               cancelCh:      make(chan uint64, 1024),
        }
        w.ctx, w.cancel = context.WithCancel(context.Background())

And the test from this PR also passes 👍

Yeah this is interesting 🤔 + the lock could be expensive.

Comment on lines 326 to 327
default:
req.enqueue <- enqueueResult{status: failed}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would add a warning here, or even an error: this shouldn't ever happen and indicates a bug.

@cyriltovena cyriltovena merged commit 37d0c6c into grafana:main Jan 13, 2022
colega added a commit to grafana/mimir that referenced this pull request Jan 13, 2022
With previous implementation, if worker was busy talking to scheduler,
we didn't push the cancellation, keeping that query running.

When cancelling a query, all its subqueries are cancelled at the same
time, so this was most likely happening all the time (first subquery
scheduled on this worker was canceled, the rest were not because worker
was busy cancelling the first one).

Also removed the `<-ctx.Done()` escape point when waiting for the
enqueueing ACK and modified the enqueueing method to ensure that it
always responds something.

Fixes: #740
Inspired by: grafana/loki#5113

Signed-off-by: Oleg Zaytsev <[email protected]>
pstibrany added a commit to grafana/mimir that referenced this pull request Jan 14, 2022
* Increase scheduler worker cancellation chan cap

With previous implementation, if worker was busy talking to scheduler,
we didn't push the cancellation, keeping that query running.

When cancelling a query, all its subqueries are cancelled at the same
time, so this was most likely happening all the time (first subquery
scheduled on this worker was canceled, the rest were not because worker
was busy cancelling the first one).

Also removed the `<-ctx.Done()` escape point when waiting for the
enqueueing ACK and modified the enqueueing method to ensure that it
always responds something.

Fixes: #740
Inspired by: grafana/loki#5113

Signed-off-by: Oleg Zaytsev <[email protected]>

* Update CHANGELOG.md

Signed-off-by: Oleg Zaytsev <[email protected]>

* Remove comment about chan memory usage

Co-authored-by: Peter Štibraný <[email protected]>

* Update test comment

Co-authored-by: Peter Štibraný <[email protected]>

* Add resp.Error to the log when response is unknown

Signed-off-by: Oleg Zaytsev <[email protected]>

* Log the entire uknown response

Signed-off-by: Oleg Zaytsev <[email protected]>

Co-authored-by: Peter Štibraný <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Request cancel issue between loki QueryFrontend and QueryFrontendWorker
3 participants