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

Gracefully shutdown querier when using query-scheduler #1756

Merged
merged 9 commits into from
Apr 27, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
* [ENHANCEMENT] Admin: Admin API now has some styling. #1482 #1549
* [ENHANCEMENT] Alertmanager: added `insight=true` field to alertmanager dispatch logs. #1379
* [ENHANCEMENT] Store-gateway: Add the experimental ability to run index header operations in a dedicated thread pool. This feature can be configured using `-blocks-storage.bucket-store.index-header-thread-pool-size` and is disabled by default. #1660
* [ENHANCEMENT] Querier: wait until inflight queries are completed when shutting down queriers and running Mimir with query-scheduler. #1756
* [BUGFIX] Query-frontend: do not shard queries with a subquery unless the subquery is inside a shardable aggregation function call. #1542
* [BUGFIX] Query-frontend: added `component=query-frontend` label to results cache memcached metrics to fix a panic when Mimir is running in single binary mode and results cache is enabled. #1704
* [BUGFIX] Mimir: services' status content-type is now correctly set to `text/html`. #1575
Expand Down
29 changes: 22 additions & 7 deletions pkg/querier/worker/scheduler_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,10 @@ func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, r
querierID: cfg.QuerierID,
grpcConfig: cfg.GRPCClientConfig,

schedulerClientFactory: func(conn *grpc.ClientConn) schedulerpb.SchedulerForQuerierClient {
return schedulerpb.NewSchedulerForQuerierClient(conn)
},

frontendClientRequestDuration: promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{
Name: "cortex_querier_query_frontend_request_duration_seconds",
Help: "Time spend doing requests to frontend.",
Expand Down Expand Up @@ -75,11 +79,13 @@ type schedulerProcessor struct {

frontendPool *client.Pool
frontendClientRequestDuration *prometheus.HistogramVec

schedulerClientFactory func(conn *grpc.ClientConn) schedulerpb.SchedulerForQuerierClient
}

// notifyShutdown implements processor.
func (sp *schedulerProcessor) notifyShutdown(ctx context.Context, conn *grpc.ClientConn, address string) {
client := schedulerpb.NewSchedulerForQuerierClient(conn)
client := sp.schedulerClientFactory(conn)

req := &schedulerpb.NotifyQuerierShutdownRequest{QuerierID: sp.querierID}
if _, err := client.NotifyQuerierShutdown(ctx, req); err != nil {
Expand All @@ -88,12 +94,17 @@ func (sp *schedulerProcessor) notifyShutdown(ctx context.Context, conn *grpc.Cli
}
}

func (sp *schedulerProcessor) processQueriesOnSingleStream(ctx context.Context, conn *grpc.ClientConn, address string) {
schedulerClient := schedulerpb.NewSchedulerForQuerierClient(conn)
func (sp *schedulerProcessor) processQueriesOnSingleStream(workerCtx context.Context, conn *grpc.ClientConn, address string) {
schedulerClient := sp.schedulerClientFactory(conn)

// Run the querier loop (and so all the queries) in a dedicated context that we call the "execution context".
// The execution context is cancelled once the workerCtx is cancelled AND there's no inflight query executing.
exec := newExecutionContext(workerCtx, sp.log)
defer exec.cancel()

backoff := backoff.New(ctx, processorBackoffConfig)
backoff := backoff.New(exec.context(), processorBackoffConfig)
for backoff.Ongoing() {
c, err := schedulerClient.QuerierLoop(ctx)
c, err := schedulerClient.QuerierLoop(exec.context())
if err == nil {
err = c.Send(&schedulerpb.QuerierToScheduler{QuerierID: sp.querierID})
}
Expand All @@ -104,7 +115,7 @@ func (sp *schedulerProcessor) processQueriesOnSingleStream(ctx context.Context,
continue
}

if err := sp.querierLoop(c, address); err != nil {
if err := sp.querierLoop(c, address, exec); err != nil {
level.Error(sp.log).Log("msg", "error processing requests from scheduler", "err", err, "addr", address)
backoff.Wait()
continue
Expand All @@ -115,7 +126,7 @@ func (sp *schedulerProcessor) processQueriesOnSingleStream(ctx context.Context,
}

// process loops processing requests on an established stream.
func (sp *schedulerProcessor) querierLoop(c schedulerpb.SchedulerForQuerier_QuerierLoopClient, address string) error {
func (sp *schedulerProcessor) querierLoop(c schedulerpb.SchedulerForQuerier_QuerierLoopClient, address string, exec *executionContext) error {
// Build a child context so we can cancel a query when the stream is closed.
ctx, cancel := context.WithCancel(c.Context())
defer cancel()
Expand All @@ -126,12 +137,16 @@ func (sp *schedulerProcessor) querierLoop(c schedulerpb.SchedulerForQuerier_Quer
return err
}

exec.queryStarted()

// Handle the request on a "background" goroutine, so we go back to
// blocking on c.Recv(). This allows us to detect the stream closing
// and cancel the query. We don't actually handle queries in parallel
// here, as we're running in lock step with the server - each Recv is
// paired with a Send.
go func() {
defer exec.queryEnded()

// We need to inject user into context for sending response back.
ctx := user.InjectOrgID(ctx, request.UserID)

Expand Down
197 changes: 197 additions & 0 deletions pkg/querier/worker/scheduler_processor_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,197 @@
// SPDX-License-Identifier: AGPL-3.0-only

package worker

import (
"context"
"testing"
"time"

"github.com/go-kit/log"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/httpgrpc"
"go.uber.org/atomic"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"

"github.com/grafana/mimir/pkg/scheduler/schedulerpb"
)

func TestSchedulerProcessor_processQueriesOnSingleStream(t *testing.T) {
t.Run("should immediately return if worker context is canceled and there's no inflight query", func(t *testing.T) {
sp, loopClient, requestHandler := prepareSchedulerProcessor()

loopClient.On("Recv").Return(func() (*schedulerpb.SchedulerToQuerier, error) {
// No query to execute, so wait until terminated.
<-loopClient.Context().Done()
return nil, loopClient.Context().Err()
})

requestHandler.On("Handle", mock.Anything, mock.Anything).Return(&httpgrpc.HTTPResponse{}, nil)

workerCtx, workerCancel := context.WithCancel(context.Background())
workerCancel()

sp.processQueriesOnSingleStream(workerCtx, nil, "127.0.0.1")

// We expect at this point, the execution context has been canceled too.
require.Error(t, loopClient.Context().Err())

// We expect Send() has been called only once, to send the querier ID to scheduler.
loopClient.AssertNumberOfCalls(t, "Send", 1)
loopClient.AssertCalled(t, "Send", &schedulerpb.QuerierToScheduler{QuerierID: "test-querier-id"})

})

t.Run("should wait until inflight query execution is completed before returning when worker context is canceled", func(t *testing.T) {
sp, loopClient, requestHandler := prepareSchedulerProcessor()

recvCount := atomic.NewInt64(0)

loopClient.On("Recv").Return(func() (*schedulerpb.SchedulerToQuerier, error) {
switch recvCount.Inc() {
case 1:
return &schedulerpb.SchedulerToQuerier{
QueryID: 1,
HttpRequest: nil,
FrontendAddress: "127.0.0.2",
UserID: "user-1",
}, nil
default:
// No more messages to process, so waiting until terminated.
<-loopClient.Context().Done()
return nil, loopClient.Context().Err()
}
})

workerCtx, workerCancel := context.WithCancel(context.Background())

requestHandler.On("Handle", mock.Anything, mock.Anything).Run(func(args mock.Arguments) {
// Cancel the worker context while the query execution is in progress.
workerCancel()

// Ensure the execution context hasn't been canceled yet.
require.Nil(t, loopClient.Context().Err())

// Intentionally slow down the query execution, to double check the worker waits until done.
time.Sleep(time.Second)
}).Return(&httpgrpc.HTTPResponse{}, nil)

startTime := time.Now()
sp.processQueriesOnSingleStream(workerCtx, nil, "127.0.0.1")
assert.GreaterOrEqual(t, time.Since(startTime), time.Second)

// We expect at this point, the execution context has been canceled too.
require.Error(t, loopClient.Context().Err())

// We expect Send() to be called twice: first to send the querier ID to scheduler
// and then to send the query result.
loopClient.AssertNumberOfCalls(t, "Send", 2)
loopClient.AssertCalled(t, "Send", &schedulerpb.QuerierToScheduler{QuerierID: "test-querier-id"})
})
}

func prepareSchedulerProcessor() (*schedulerProcessor, *querierLoopClientMock, *requestHandlerMock) {
var querierLoopCtx context.Context

loopClient := &querierLoopClientMock{}
loopClient.On("Send", mock.Anything).Return(nil)
loopClient.On("Context").Return(func() context.Context {
return querierLoopCtx
})

schedulerClient := &schedulerForQuerierClientMock{}
schedulerClient.On("QuerierLoop", mock.Anything, mock.Anything).Run(func(args mock.Arguments) {
querierLoopCtx = args.Get(0).(context.Context)
}).Return(loopClient, nil)

requestHandler := &requestHandlerMock{}

sp, _ := newSchedulerProcessor(Config{QuerierID: "test-querier-id"}, requestHandler, log.NewNopLogger(), nil)
sp.schedulerClientFactory = func(_ *grpc.ClientConn) schedulerpb.SchedulerForQuerierClient {
return schedulerClient
}

return sp, loopClient, requestHandler
}

type schedulerForQuerierClientMock struct {
mock.Mock
}

func (m *schedulerForQuerierClientMock) QuerierLoop(ctx context.Context, opts ...grpc.CallOption) (schedulerpb.SchedulerForQuerier_QuerierLoopClient, error) {
args := m.Called(ctx, opts)
return args.Get(0).(schedulerpb.SchedulerForQuerier_QuerierLoopClient), args.Error(1)
}

func (m *schedulerForQuerierClientMock) NotifyQuerierShutdown(ctx context.Context, in *schedulerpb.NotifyQuerierShutdownRequest, opts ...grpc.CallOption) (*schedulerpb.NotifyQuerierShutdownResponse, error) {
args := m.Called(ctx, in, opts)
return args.Get(0).(*schedulerpb.NotifyQuerierShutdownResponse), args.Error(1)
}

type querierLoopClientMock struct {
mock.Mock
}

func (m *querierLoopClientMock) Send(msg *schedulerpb.QuerierToScheduler) error {
args := m.Called(msg)
return args.Error(0)
}

func (m *querierLoopClientMock) Recv() (*schedulerpb.SchedulerToQuerier, error) {
args := m.Called()

// Allow to mock the Recv() with a function which is called each time.
if fn, ok := args.Get(0).(func() (*schedulerpb.SchedulerToQuerier, error)); ok {
return fn()
}

return args.Get(0).(*schedulerpb.SchedulerToQuerier), args.Error(1)
}

func (m *querierLoopClientMock) Header() (metadata.MD, error) {
args := m.Called()
return args.Get(0).(metadata.MD), args.Error(1)
}

func (m *querierLoopClientMock) Trailer() metadata.MD {
args := m.Called()
return args.Get(0).(metadata.MD)
}

func (m *querierLoopClientMock) CloseSend() error {
args := m.Called()
return args.Error(0)
}

func (m *querierLoopClientMock) Context() context.Context {
args := m.Called()

// Allow to mock the Context() with a function which is called each time.
if fn, ok := args.Get(0).(func() context.Context); ok {
return fn()
}

return args.Get(0).(context.Context)
}

func (m *querierLoopClientMock) SendMsg(msg interface{}) error {
args := m.Called(msg)
return args.Error(0)
}

func (m *querierLoopClientMock) RecvMsg(msg interface{}) error {
args := m.Called(msg)
return args.Error(0)
}

type requestHandlerMock struct {
mock.Mock
}

func (m *requestHandlerMock) Handle(ctx context.Context, req *httpgrpc.HTTPRequest) (*httpgrpc.HTTPResponse, error) {
args := m.Called(ctx, req)
return args.Get(0).(*httpgrpc.HTTPResponse), args.Error(1)
}
83 changes: 83 additions & 0 deletions pkg/querier/worker/util.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// SPDX-License-Identifier: AGPL-3.0-only

package worker

import (
"context"
"time"

"github.com/go-kit/log"
"github.com/go-kit/log/level"
"go.uber.org/atomic"
)

// executionContext wraps the context.Context used to run the querier's worker loop and execute
// queries. The purpose of the execution context is to gracefully shutdown queriers, waiting
// until inflight queries are terminated before the querier process exits.
//
// How it's used:
//
// - The querier worker's loop run in a dedicated context, called the "execution context".
//
// - The execution context is canceled when the worker context gets cancelled (ie. querier is shutting down)
// and there's no inflight query execution. In case there's an inflight query, the execution context is canceled
// once the inflight query terminates and the response has been sent.
type executionContext struct {
execCtx context.Context
execCancel context.CancelFunc
inflightQuery *atomic.Bool
}

// newExecutionContext returns a new executionContext. The caller must call cancel() on it once done.
func newExecutionContext(workerCtx context.Context, logger log.Logger) *executionContext {
Copy link
Member

Choose a reason for hiding this comment

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

Nit: It's not clear that one needs to call cancel() on returned execution context (documentation missing). Returning ctx, cancelFn, *atomic.Bool() directly (without wrapping to the struct) would be more consistent with existing context-related functions, and more explicit that cancelFn must be called.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I added a comment on newExecutionContext(). About removing the struct I will see how I will end up fixing the same for the case Mimir is running without query-scheduler first.

execCtx, execCancel := context.WithCancel(context.Background())

c := &executionContext{
execCtx: execCtx,
execCancel: execCancel,
inflightQuery: atomic.NewBool(false),
}

go func() {
// Wait until it's safe to cancel the execution context, which is when one of the following conditions happen:
// - The worker context has been canceled and there's no inflight query
// - The execution context itself has been explicitly canceled
select {
case <-workerCtx.Done():
level.Debug(logger).Log("msg", "querier worker context has been canceled, waiting until there's no inflight query")

for c.inflightQuery.Load() {
select {
case <-execCtx.Done():
// In the meanwhile, the execution context has been explicitly canceled, so we should just terminate.
return
case <-time.After(100 * time.Millisecond):
// Going to check it again.
}
}

level.Debug(logger).Log("msg", "querier worker context has been canceled and there's no inflight query, canceling the execution context too")
execCancel()
case <-execCtx.Done():
// Nothing to do. The execution context has been explicitly canceled.
}
}()

return c
}

func (c *executionContext) queryStarted() {
c.inflightQuery.Store(true)
}

func (c *executionContext) queryEnded() {
c.inflightQuery.Store(false)
}

func (c *executionContext) context() context.Context {
return c.execCtx
}

func (c *executionContext) cancel() {
c.execCancel()
}
6 changes: 6 additions & 0 deletions pkg/scheduler/queue/user_queues.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,12 @@ func (q *queues) getOrAddQueue(userID string, maxQueriers int) chan Request {
func (q *queues) getNextQueueForQuerier(lastUserIndex int, querierID string) (chan Request, string, int) {
uid := lastUserIndex

// Ensure the querier is not shutting down. If the querier is shutting down, we shouldn't forward
// any more queries to it.
if info := q.queriers[querierID]; info == nil || info.shuttingDown {
return nil, "", uid
}

for iters := 0; iters < len(q.users); iters++ {
uid = uid + 1

Expand Down
Loading