-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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: Mitigate ingester race between Query & GetChunkIDs #15178
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -35,6 +35,10 @@ import ( | |
util_log "github.com/grafana/loki/v3/pkg/util/log" | ||
) | ||
|
||
var defaultQuorumConfig = ring.DoUntilQuorumConfig{ | ||
// Nothing here | ||
} | ||
|
||
type responseFromIngesters struct { | ||
addr string | ||
response interface{} | ||
|
@@ -79,7 +83,8 @@ func newIngesterQuerier(querierConfig Config, clientCfg client.Config, ring ring | |
} | ||
|
||
// forAllIngesters runs f, in parallel, for all ingesters | ||
func (q *IngesterQuerier) forAllIngesters(ctx context.Context, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { | ||
// waitForAllResponses param can be used to require results from all ingesters in the replication set. If this is set to false, the call will return as soon as we have a quorum by zone. Only valid for partition-ingesters. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we can improve this comment, let's discuss offline. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. As discussed offline, we will follow up with some better comments in another PR. |
||
func (q *IngesterQuerier) forAllIngesters(ctx context.Context, waitForAllResponses bool, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { | ||
if q.querierConfig.QueryPartitionIngesters { | ||
tenantID, err := user.ExtractOrgID(ctx) | ||
if err != nil { | ||
|
@@ -94,36 +99,36 @@ func (q *IngesterQuerier) forAllIngesters(ctx context.Context, f func(context.Co | |
if err != nil { | ||
return nil, err | ||
} | ||
return q.forGivenIngesterSets(ctx, replicationSets, f) | ||
return q.forGivenIngesterSets(ctx, waitForAllResponses, replicationSets, f) | ||
} | ||
|
||
replicationSet, err := q.ring.GetReplicationSetForOperation(ring.Read) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
return q.forGivenIngesters(ctx, replicationSet, defaultQuorumConfig(), f) | ||
return q.forGivenIngesters(ctx, replicationSet, defaultQuorumConfig, f) | ||
} | ||
|
||
// forGivenIngesterSets runs f, in parallel, for given ingester sets | ||
func (q *IngesterQuerier) forGivenIngesterSets(ctx context.Context, replicationSet []ring.ReplicationSet, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { | ||
// Enable minimize requests so we initially query a single ingester per replication set, as each replication-set is one partition. | ||
// waitForAllResponses param can be used to require results from all ingesters in all replication sets. If this is set to false, the call will return as soon as we have a quorum by zone. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same here. Let's see if we can improve this comment. If I was unfamiliar with how this works, I would be surprised because |
||
func (q *IngesterQuerier) forGivenIngesterSets(ctx context.Context, waitForAllResponses bool, replicationSet []ring.ReplicationSet, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { | ||
// Enable minimize requests if we can, so we initially query a single ingester per replication set, as each replication-set is one partition. | ||
// Ingesters must supply zone information for this to have an effect. | ||
config := ring.DoUntilQuorumConfig{ | ||
MinimizeRequests: true, | ||
MinimizeRequests: !waitForAllResponses, | ||
} | ||
return concurrency.ForEachJobMergeResults[ring.ReplicationSet, responseFromIngesters](ctx, replicationSet, 0, func(ctx context.Context, set ring.ReplicationSet) ([]responseFromIngesters, error) { | ||
if waitForAllResponses { | ||
// Tell the ring we need to return all responses from all zones | ||
set.MaxErrors = 0 | ||
set.MaxUnavailableZones = 0 | ||
} | ||
return q.forGivenIngesters(ctx, set, config, f) | ||
}) | ||
} | ||
|
||
func defaultQuorumConfig() ring.DoUntilQuorumConfig { | ||
return ring.DoUntilQuorumConfig{ | ||
// Nothing here | ||
} | ||
} | ||
|
||
// forGivenIngesters runs f, in parallel, for given ingesters | ||
// forGivenIngesters runs f, in parallel, for given ingesters until a quorum of responses are received | ||
func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet ring.ReplicationSet, quorumConfig ring.DoUntilQuorumConfig, f func(context.Context, logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) { | ||
results, err := ring.DoUntilQuorum(ctx, replicationSet, quorumConfig, func(ctx context.Context, ingester *ring.InstanceDesc) (responseFromIngesters, error) { | ||
client, err := q.pool.GetClientFor(ingester.Addr) | ||
|
@@ -152,7 +157,7 @@ func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet | |
} | ||
|
||
func (q *IngesterQuerier) SelectLogs(ctx context.Context, params logql.SelectLogParams) ([]iter.EntryIterator, error) { | ||
resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
stats.FromContext(ctx).AddIngesterReached(1) | ||
return client.Query(ctx, params.QueryRequest) | ||
}) | ||
|
@@ -168,7 +173,7 @@ func (q *IngesterQuerier) SelectLogs(ctx context.Context, params logql.SelectLog | |
} | ||
|
||
func (q *IngesterQuerier) SelectSample(ctx context.Context, params logql.SelectSampleParams) ([]iter.SampleIterator, error) { | ||
resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
stats.FromContext(ctx).AddIngesterReached(1) | ||
return client.QuerySample(ctx, params.SampleQueryRequest) | ||
}) | ||
|
@@ -184,7 +189,7 @@ func (q *IngesterQuerier) SelectSample(ctx context.Context, params logql.SelectS | |
} | ||
|
||
func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) ([][]string, error) { | ||
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
return client.Label(ctx, req) | ||
}) | ||
if err != nil { | ||
|
@@ -200,7 +205,7 @@ func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) | |
} | ||
|
||
func (q *IngesterQuerier) Tail(ctx context.Context, req *logproto.TailRequest) (map[string]logproto.Querier_TailClient, error) { | ||
resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
return client.Tail(ctx, req) | ||
}) | ||
if err != nil { | ||
|
@@ -249,7 +254,7 @@ func (q *IngesterQuerier) TailDisconnectedIngesters(ctx context.Context, req *lo | |
} | ||
|
||
// Instance a tail client for each ingester to re(connect) | ||
reconnectClients, err := q.forGivenIngesters(ctx, ring.ReplicationSet{Instances: reconnectIngesters}, defaultQuorumConfig(), func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
reconnectClients, err := q.forGivenIngesters(ctx, ring.ReplicationSet{Instances: reconnectIngesters}, defaultQuorumConfig, func(_ context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
return client.Tail(ctx, req) | ||
}) | ||
if err != nil { | ||
|
@@ -265,7 +270,7 @@ func (q *IngesterQuerier) TailDisconnectedIngesters(ctx context.Context, req *lo | |
} | ||
|
||
func (q *IngesterQuerier) Series(ctx context.Context, req *logproto.SeriesRequest) ([][]logproto.SeriesIdentifier, error) { | ||
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
return client.Series(ctx, req) | ||
}) | ||
if err != nil { | ||
|
@@ -297,7 +302,7 @@ func (q *IngesterQuerier) TailersCount(ctx context.Context) ([]uint32, error) { | |
return nil, httpgrpc.Errorf(http.StatusInternalServerError, "no active ingester found") | ||
} | ||
|
||
responses, err := q.forGivenIngesters(ctx, replicationSet, defaultQuorumConfig(), func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
responses, err := q.forGivenIngesters(ctx, replicationSet, defaultQuorumConfig, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
resp, err := querierClient.TailersCount(ctx, &logproto.TailersCountRequest{}) | ||
if err != nil { | ||
return nil, err | ||
|
@@ -320,7 +325,9 @@ func (q *IngesterQuerier) TailersCount(ctx context.Context) ([]uint32, error) { | |
} | ||
|
||
func (q *IngesterQuerier) GetChunkIDs(ctx context.Context, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { | ||
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
// We must wait for all responses when using partition-ingesters to avoid a race between Query and GetChunkIDs calls. | ||
// This occurs if call Query on an ingester after a recent flush then call GetChunkIDs on a different, unflushed ingester in the same partition. | ||
resps, err := q.forAllIngesters(ctx, q.querierConfig.QueryPartitionIngesters, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
return querierClient.GetChunkIDs(ctx, &logproto.GetChunkIDsRequest{ | ||
Matchers: convertMatchersToString(matchers), | ||
Start: from.Time(), | ||
|
@@ -340,7 +347,7 @@ func (q *IngesterQuerier) GetChunkIDs(ctx context.Context, from, through model.T | |
} | ||
|
||
func (q *IngesterQuerier) Stats(ctx context.Context, _ string, from, through model.Time, matchers ...*labels.Matcher) (*index_stats.Stats, error) { | ||
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
return querierClient.GetStats(ctx, &logproto.IndexStatsRequest{ | ||
From: from, | ||
Through: through, | ||
|
@@ -371,7 +378,7 @@ func (q *IngesterQuerier) Volume(ctx context.Context, _ string, from, through mo | |
matcherString = syntax.MatchersString(matchers) | ||
} | ||
|
||
resps, err := q.forAllIngesters(ctx, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
resps, err := q.forAllIngesters(ctx, false, func(ctx context.Context, querierClient logproto.QuerierClient) (interface{}, error) { | ||
return querierClient.GetVolume(ctx, &logproto.VolumeRequest{ | ||
From: from, | ||
Through: through, | ||
|
@@ -400,7 +407,7 @@ func (q *IngesterQuerier) Volume(ctx context.Context, _ string, from, through mo | |
} | ||
|
||
func (q *IngesterQuerier) DetectedLabel(ctx context.Context, req *logproto.DetectedLabelsRequest) (*logproto.LabelToValuesResponse, error) { | ||
ingesterResponses, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
ingesterResponses, err := q.forAllIngesters(ctx, false, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { | ||
return client.GetDetectedLabels(ctx, req) | ||
}) | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think redundant comment. If we do want a comment I would write something like this (finishing the sentence):