Skip to content

Commit

Permalink
Fix cancel issue between Query Frontend and Query Schdeduler (#5113)
Browse files Browse the repository at this point in the history
* Fix cancel issue between Query Frontend -> Query Schdeduler

Signed-off-by: Kaviraj <[email protected]>

* remove assert check for message type

Signed-off-by: Kaviraj <[email protected]>

* Handle the case where Frontend is running without Frontend workers

Signed-off-by: Kaviraj <[email protected]>

* Make it into `sendRequestCancel()` func

Signed-off-by: Kaviraj <[email protected]>

* Remove debug logs

Signed-off-by: Kaviraj <[email protected]>

* Fix note

Signed-off-by: Kaviraj <[email protected]>

* Remove workers count debug log on test

Signed-off-by: Kaviraj <[email protected]>

* Add bug link to test

Signed-off-by: Kaviraj <[email protected]>

* Fixes the case where we cancel after enqueuing

Signed-off-by: Cyril Tovena <[email protected]>

* Update pkg/lokifrontend/frontend/v2/frontend.go

Co-authored-by: Oleg Zaytsev <[email protected]>

* Switch to a buffered channel instead

Signed-off-by: Cyril Tovena <[email protected]>

* Add error on unknown response status from scheduler

Signed-off-by: Kaviraj <[email protected]>

Co-authored-by: Cyril Tovena <[email protected]>
Co-authored-by: Oleg Zaytsev <[email protected]>
  • Loading branch information
3 people authored Jan 13, 2022
1 parent 7ff1b45 commit 37d0c6c
Show file tree
Hide file tree
Showing 3 changed files with 46 additions and 9 deletions.
11 changes: 3 additions & 8 deletions pkg/lokifrontend/frontend/v2/frontend.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,21 +192,15 @@ func (f *Frontend) RoundTripGRPC(ctx context.Context, req *httpgrpc.HTTPRequest)
retries := f.cfg.WorkerConcurrency + 1 // To make sure we hit at least two different schedulers.

enqueueAgain:
var cancelCh chan<- uint64
select {
case <-ctx.Done():
return nil, ctx.Err()

case f.requestsCh <- freq:
// Enqueued, let's wait for response.
}

var cancelCh chan<- uint64

select {
case <-ctx.Done():
return nil, ctx.Err()
enqRes := <-freq.enqueue

case enqRes := <-freq.enqueue:
if enqRes.status == waitForResponse {
cancelCh = enqRes.cancelCh
break // go wait for response.
Expand All @@ -228,6 +222,7 @@ enqueueAgain:
// cancellation sent.
default:
// failed to cancel, ignore.
level.Warn(f.log).Log("msg", "failed to send cancellation request to scheduler, queue full")
}
}
return nil, ctx.Err()
Expand Down
6 changes: 5 additions & 1 deletion pkg/lokifrontend/frontend/v2/frontend_scheduler_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,8 @@ func newFrontendSchedulerWorker(conn *grpc.ClientConn, schedulerAddr string, fro
schedulerAddr: schedulerAddr,
frontendAddr: frontendAddr,
requestCh: requestCh,
cancelCh: make(chan uint64),
// Allow to enqueue enough cancellation requests. ~ 8MB memory size.
cancelCh: make(chan uint64, 1000000),
}
w.ctx, w.cancel = context.WithCancel(context.Background())

Expand Down Expand Up @@ -322,6 +323,9 @@ func (w *frontendSchedulerWorker) schedulerLoop(loop schedulerpb.SchedulerForFro
Body: []byte("too many outstanding requests"),
},
}
default:
level.Error(w.log).Log("msg", "unknown response status from the scheduler", "status", resp.Status, "queryID", req.queryID)
req.enqueue <- enqueueResult{status: failed}
}

case reqID := <-w.cancelCh:
Expand Down
38 changes: 38 additions & 0 deletions pkg/lokifrontend/frontend/v2/frontend_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,44 @@ func TestFrontendCancellation(t *testing.T) {
})
}

// If FrontendWorkers are busy, cancellation passed by Query frontend may not reach
// all the frontend workers thus not reaching the scheduler as well.
// Issue: https://github.com/grafana/loki/issues/5132
func TestFrontendWorkerCancellation(t *testing.T) {
f, ms := setupFrontend(t, nil)

ctx, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond)
defer cancel()

// send multiple requests > maxconcurrency of scheduler. So that it keeps all the frontend worker busy in serving requests.
reqCount := testFrontendWorkerConcurrency + 5
var wg sync.WaitGroup
for i := 0; i < reqCount; i++ {
wg.Add(1)
go func() {
defer wg.Done()
resp, err := f.RoundTripGRPC(user.InjectOrgID(ctx, "test"), &httpgrpc.HTTPRequest{})
require.EqualError(t, err, context.DeadlineExceeded.Error())
require.Nil(t, resp)
}()
}

wg.Wait()

// We wait a bit to make sure scheduler receives the cancellation request.
// 2 * reqCount because for every request, should also be corresponding cancel request
test.Poll(t, 5*time.Second, 2*reqCount, func() interface{} {
ms.mu.Lock()
defer ms.mu.Unlock()

return len(ms.msgs)
})

ms.checkWithLock(func() {
require.Equal(t, 2*reqCount, len(ms.msgs))
})
}

func TestFrontendFailedCancellation(t *testing.T) {
f, ms := setupFrontend(t, nil)

Expand Down

0 comments on commit 37d0c6c

Please sign in to comment.