Move all intrinsic tag lookup to the query-frontend and prioritize them in the results (#4784)

* Move all intrinsic tag lookup to the query-frontend and prioritize them in the results

* lint, error handling, fix tests

* Fix some tests

* Revert unintended change to search/tags v1 behavior, update tests

* Reduce diff

* Revert unintended change to 'none' scope

* reduce diff

* changelog

* Update test to test intrinsic handling at the limit

* todos
This commit is contained in:
Martin Disibio
2025-03-04 15:12:03 -05:00
committed by GitHub
parent e21bce7593
commit 5d540e119f
12 changed files with 206 additions and 84 deletions

View File

@ -57,6 +57,7 @@ configurable via the throughput_bytes_slo field, and it will populate op="traces
* [BUGFIX] Fix memcached settings for docker compose example [#4346](https://github.com/grafana/tempo/pull/4695) (@ruslan-mikhailov)
* [BUGFIX] Fix setting processors in user configurations overrides via API [#4741](https://github.com/grafana/tempo/pull/4741) (@ruslan-mikhailov)
* [BUGFIX] Fix panic on startup [#4744](https://github.com/grafana/tempo/pull/4744) (@ruslan-mikhailov)
* [BUGFIX] Fix intrinsic tag lookups dropped when max tag lookup response size is exceeded [#4784](https://github.com/grafana/tempo/pull/4784) (@mdisibio)
* [BUGFIX] Correctly cache frontend jobs for query range (TraceQL Metrics). [#4771](https://github.com/grafana/tempo/pull/4771) (@joe-elliott)
# v2.7.1

View File

@ -16,6 +16,7 @@ import (
"github.com/grafana/e2e"
"github.com/grafana/tempo/integration/util"
"github.com/grafana/tempo/pkg/search"
"github.com/grafana/tempo/pkg/tempopb"
"github.com/stretchr/testify/require"
"google.golang.org/grpc/codes"
@ -652,11 +653,12 @@ func callSearchTagValuesV2AndAssert(t *testing.T, svc *e2e.HTTPService, tagName,
func callSearchTagsV2AndAssert(t *testing.T, svc *e2e.HTTPService, scope, query string, expected searchTagsV2Response, start, end int64) {
urlPath := fmt.Sprintf(`/api/v2/search/tags?scope=%s&q=%s`, scope, url.QueryEscape(query))
// expected will not have the intrinsic scope since it's the same every time, add it here.
// Expected will not have the intrinsic results to make the tests simpler,
// they are added here based on the scope.
if scope == "none" || scope == "" || scope == "intrinsic" {
expected.Scopes = append(expected.Scopes, ScopedTags{
Name: "intrinsic",
Tags: []string{"duration", "event:name", "event:timeSinceStart", "instrumentation:name", "instrumentation:version", "kind", "name", "rootName", "rootServiceName", "span:duration", "span:kind", "span:name", "span:status", "span:statusMessage", "status", "statusMessage", "trace:duration", "trace:rootName", "trace:rootService", "traceDuration"},
Tags: search.GetVirtualIntrinsicValues(),
})
}
sort.Slice(expected.Scopes, func(i, j int) bool { return expected.Scopes[i].Name < expected.Scopes[j].Name })
@ -776,9 +778,7 @@ func callSearchTagsAndAssert(t *testing.T, svc *e2e.HTTPService, expected search
// parse response
var response searchTagsResponse
require.NoError(t, json.Unmarshal(body, &response))
sort.Strings(response.TagNames)
sort.Strings(expected.TagNames)
require.Equal(t, expected.TagNames, response.TagNames)
require.ElementsMatch(t, expected.TagNames, response.TagNames)
assertMetrics(t, response.Metrics, len(response.TagNames))
// streaming
@ -808,8 +808,8 @@ func callSearchTagsAndAssert(t *testing.T, svc *e2e.HTTPService, expected search
if grpcResp.TagNames == nil {
grpcResp.TagNames = []string{}
}
sort.Slice(grpcResp.TagNames, func(i, j int) bool { return grpcResp.TagNames[i] < grpcResp.TagNames[j] })
require.Equal(t, expected.TagNames, grpcResp.TagNames)
require.ElementsMatch(t, expected.TagNames, grpcResp.TagNames)
// assert metrics, and make sure it's non-zero when response is non-empty
if len(grpcResp.TagNames) > 0 {
require.Greater(t, grpcResp.Metrics.InspectedBytes, uint64(100))

View File

@ -476,12 +476,12 @@ func SearchAndAssertTraceBackend(t *testing.T, client *httpclient.Client, info *
// by passing a time range and using a query_ingesters_until/backend_after of 0 we can force the queriers
// to look in the backend blocks
func SearchAndAsserTagsBackend(t *testing.T, client *httpclient.Client, start, end int64) {
// There are no tags in recent data
resp, err := client.SearchTags()
require.NoError(t, err)
require.Equal(t, len(resp.TagNames), 0)
// verify trace can be found using attribute and time range
// There are additional tags in the backend
resp, err = client.SearchTagsWithRange(start, end)
require.NoError(t, err)
require.True(t, len(resp.TagNames) > 0)

View File

@ -137,6 +137,13 @@ func (c *genericCombiner[T]) AddResponse(r PipelineResponse) error {
return nil
}
func (c *genericCombiner[T]) AddTypedResponse(r T) error {
c.mu.Lock()
defer c.mu.Unlock()
return c.combine(r, c.current, nil)
}
// HTTPFinal, GRPCComplete, and GRPCDiff are all responsible for returning something
// usable in grpc streaming/http response.
// NOTE: returning error is reserved for unexpected errors, HTTP errors will be returned

View File

@ -17,8 +17,13 @@ type Combiner interface {
HTTPFinal() (*http.Response, error)
}
type TypedCombiner[T TResponse] interface {
AddTypedResponse(r T) error
}
type GRPCCombiner[T TResponse] interface {
Combiner
TypedCombiner[T]
GRPCFinal() (T, error)
GRPCDiff() (T, error)

View File

@ -13,6 +13,7 @@ import (
"github.com/gogo/protobuf/jsonpb"
"github.com/gogo/protobuf/proto"
"github.com/grafana/dskit/user"
"github.com/grafana/tempo/modules/overrides"
"github.com/grafana/tempo/pkg/api"
"github.com/grafana/tempo/pkg/cache"
"github.com/grafana/tempo/pkg/tempopb"
@ -52,7 +53,7 @@ func TestQueryRangeHandlerSucceeds(t *testing.T) {
responseFn: func() proto.Message {
return resp
},
}, nil, nil, nil, func(c *Config) {
}, nil, nil, nil, func(c *Config, _ *overrides.Config) {
c.Metrics.Sharder.Interval = time.Hour
})

View File

@ -718,7 +718,7 @@ func BenchmarkSearchPipeline(b *testing.B) {
// frontendWithSettings returns a new frontend with the given settings. any nil options
// are given "happy path" defaults
func frontendWithSettings(t require.TestingT, next pipeline.RoundTripper, rdr tempodb.Reader, cfg *Config, cacheProvider cache.Provider,
opts ...func(*Config),
opts ...func(*Config, *overrides.Config),
) *QueryFrontend {
if next == nil {
next = &mockRoundTripper{
@ -802,11 +802,13 @@ func frontendWithSettings(t require.TestingT, next pipeline.RoundTripper, rdr te
}
}
overridesCfg := &overrides.Config{}
for _, o := range opts {
o(cfg)
o(cfg, overridesCfg)
}
o, err := overrides.NewOverrides(overrides.Config{}, nil, prometheus.DefaultRegisterer)
o, err := overrides.NewOverrides(*overridesCfg, nil, prometheus.DefaultRegisterer)
require.NoError(t, err)
f, err := New(*cfg, next, o, rdr, cacheProvider, "", log.NewNopLogger(), nil)

View File

@ -21,7 +21,9 @@ import (
"github.com/grafana/tempo/modules/frontend/pipeline"
"github.com/grafana/tempo/modules/overrides"
"github.com/grafana/tempo/pkg/api"
"github.com/grafana/tempo/pkg/search"
"github.com/grafana/tempo/pkg/tempopb"
"github.com/grafana/tempo/pkg/traceql"
"google.golang.org/grpc/codes"
)
@ -53,6 +55,19 @@ func newTagsStreamingGRPCHandler(cfg Config, next pipeline.AsyncRoundTripper[com
return srv.Send(res)
})
// Add intrinsics first so that they aren't dropped by the response size limit
// NOTE - V1 tag lookup only returns intrinsics when scope is set explicitly.
if req.Scope == api.ParamScopeIntrinsic {
err := comb.AddTypedResponse(&tempopb.SearchTagsResponse{
TagNames: search.GetVirtualIntrinsicValues(),
})
if err != nil {
return err
}
// TODO: Exit early here, no need to issue more requests downstream, but some
// work needed to ensure things are still logged/metriced correctly.
}
start := time.Now()
logTagsRequest(logger, tenant, "SearchTagsStreaming", req.Scope, req.End-req.Start)
err = collector.RoundTrip(httpReq)
@ -86,6 +101,26 @@ func newTagsV2StreamingGRPCHandler(cfg Config, next pipeline.AsyncRoundTripper[c
return srv.Send(res)
})
// Add intrinsics first so that they aren't dropped by the response size limit
// NOTE - V2 tag lookup returns intrinsics for both unscoped and explicit scope requests.
if req.Scope == "" ||
req.Scope == api.ParamScopeIntrinsic ||
req.Scope == traceql.AttributeScopeNone.String() {
err := comb.AddTypedResponse(&tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
},
},
})
if err != nil {
return err
}
// TODO: For intrinsic scope only, exit early here, no need to issue more requests downstream, but some
// work needed to ensure things are still logged/metriced correctly.
}
start := time.Now()
logTagsRequest(logger, tenant, "SearchTagsV2Streaming", req.Scope, req.End-req.Start)
err = collector.RoundTrip(httpReq)
@ -190,6 +225,20 @@ func newTagsHTTPHandler(cfg Config, next pipeline.AsyncRoundTripper[combiner.Pip
scope, _, rangeDur, maxTagsPerScope, staleValueThreshold := parseParams(req)
// build and use round tripper
comb := combiner.NewTypedSearchTags(o.MaxBytesPerTagValuesQuery(tenant), maxTagsPerScope, staleValueThreshold)
// Add intrinsics first so that they aren't dropped by the response size limit
// NOTE - V1 tag lookup only returns intrinsics when scope is set explicitly.
if scope == api.ParamScopeIntrinsic {
err := comb.AddTypedResponse(&tempopb.SearchTagsResponse{
TagNames: search.GetVirtualIntrinsicValues(),
})
if err != nil {
return nil, err
}
// TODO: Exit early here, no need to issue more requests downstream, but some
// work needed to ensure things are still logged/metriced correctly.
}
rt := pipeline.NewHTTPCollector(next, cfg.ResponseConsumers, comb)
start := time.Now()
logTagsRequest(logger, tenant, "SearchTags", scope, rangeDur)
@ -224,6 +273,27 @@ func newTagsV2HTTPHandler(cfg Config, next pipeline.AsyncRoundTripper[combiner.P
scope, _, rangeDur, maxTagsPerScope, staleValueThreshold := parseParams(req)
// build and use round tripper
comb := combiner.NewTypedSearchTagsV2(o.MaxBytesPerTagValuesQuery(tenant), maxTagsPerScope, staleValueThreshold)
// Add intrinsics first so that they aren't dropped by the response size limit
// NOTE - V2 tag lookup returns intrinsics for both unscoped and explicit scope requests.
if scope == "" ||
scope == api.ParamScopeIntrinsic ||
scope == traceql.AttributeScopeNone.String() {
err := comb.AddTypedResponse(&tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
},
},
})
if err != nil {
return nil, err
}
// TODO: For intrinsic scope only, exit early here, no need to issue more requests downstream, but some
// work needed to ensure things are still logged/metriced correctly.
}
rt := pipeline.NewHTTPCollector(next, cfg.ResponseConsumers, comb)
start := time.Now()
logTagsRequest(logger, tenant, "SearchTagsV2", scope, rangeDur)

View File

@ -10,6 +10,7 @@ import (
"net/http/httptest"
"net/url"
"regexp"
"sort"
"testing"
"time"
@ -18,7 +19,10 @@ import (
"github.com/gogo/status"
"github.com/gorilla/mux"
"github.com/grafana/dskit/user"
"github.com/grafana/tempo/modules/overrides"
"github.com/grafana/tempo/pkg/api"
"github.com/grafana/tempo/pkg/cache"
"github.com/grafana/tempo/pkg/search"
"github.com/grafana/tempo/pkg/tempopb"
"github.com/grafana/tempo/pkg/util/test"
"github.com/grafana/tempo/tempodb/backend"
@ -167,6 +171,100 @@ func runnerTagValuesV2ClientCancelContext(t *testing.T, f *QueryFrontend) {
require.Equal(t, status.Error(codes.Canceled, "context canceled"), err)
}
func TestSearchTagsV2Intrinsics(t *testing.T) {
mockScope := "span"
mockTags := []string{"foo", "bar"}
tcs := []struct {
name string
maxTagBytes int
expected *tempopb.SearchTagsV2Response
}{
{
name: "unlimited",
maxTagBytes: 0,
expected: &tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
},
{
Name: mockScope,
Tags: mockTags,
},
},
},
},
{
name: "when_limited_intrinsics_first",
maxTagBytes: 100,
expected: &tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
// Only a subset of intrinsic tags will fit
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues()[0:10],
},
},
},
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
// This is the mocked data returned by querier/ingester jobs downstream.
next := &mockRoundTripper{
responseFn: func() proto.Message {
return &tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: mockScope,
Tags: mockTags,
},
},
}
},
}
f := frontendWithSettings(t, next, nil, nil, nil, func(_ *Config, overridesCfg *overrides.Config) {
overridesCfg.Defaults.Read.MaxBytesPerTagValuesQuery = tc.maxTagBytes
})
// http
httpReq := httptest.NewRequest("GET", "/api/v2/search/tags", nil)
httpResp := httptest.NewRecorder()
ctx, cancel := context.WithCancel(httpReq.Context())
defer cancel()
ctx = user.InjectOrgID(ctx, "tenant")
httpReq = httpReq.WithContext(ctx)
f.SearchTagsV2Handler.ServeHTTP(httpResp, httpReq)
require.Equal(t, http.StatusOK, httpResp.Code)
resp := &tempopb.SearchTagsV2Response{}
bytesResp, err := io.ReadAll(httpResp.Body)
require.NoError(t, err)
err = jsonpb.Unmarshal(bytes.NewReader(bytesResp), resp)
require.NoError(t, err)
// Sort scopes to give stable comparison
sort.Slice(tc.expected.Scopes, func(i, j int) bool {
return tc.expected.Scopes[i].Name < tc.expected.Scopes[j].Name
})
sort.Slice(resp.Scopes, func(i, j int) bool {
return resp.Scopes[i].Name < resp.Scopes[j].Name
})
require.Equal(t, len(tc.expected.Scopes), len(resp.Scopes))
for i := range tc.expected.Scopes {
require.ElementsMatch(t, tc.expected.Scopes[i].Tags, resp.Scopes[i].Tags)
}
})
}
}
// todo: a lot of code is replicated between all of these "failure propagates from queriers" tests. we should refactor
// to a framework that tests this against all endpoints
func TestSearchTagsV2FailurePropagatesFromQueriers(t *testing.T) {

View File

@ -17,7 +17,6 @@ import (
"github.com/grafana/tempo/pkg/api"
"github.com/grafana/tempo/pkg/boundedwaitgroup"
"github.com/grafana/tempo/pkg/collector"
"github.com/grafana/tempo/pkg/search"
"github.com/grafana/tempo/pkg/tempopb"
"github.com/grafana/tempo/pkg/traceql"
"github.com/grafana/tempo/pkg/util/log"
@ -192,11 +191,6 @@ func (i *instance) SearchTags(ctx context.Context, scope string) (*tempopb.Searc
// flatten v2 response
for _, s := range v2Response.Scopes {
// SearchTags does not include intrinsics on an empty scope, but v2 does.
if scope == "" && s.Name == api.ParamScopeIntrinsic {
continue
}
for _, t := range s.Tags {
distinctValues.Collect(t)
}
@ -219,17 +213,11 @@ func (i *instance) SearchTagsV2(ctx context.Context, req *tempopb.SearchTagsRequ
}
scope := req.Scope
// check if it's the special intrinsic scope
if scope == api.ParamScopeIntrinsic {
return &tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
},
},
Metrics: &tempopb.MetadataMetrics{InspectedBytes: 0}, // no bytes read for intrinsics
}, nil
// For the intrinsic scope there is nothing to do in the ingester,
// these are always added by the frontend.
return &tempopb.SearchTagsV2Response{}, nil
}
// parse for normal scopes
@ -319,14 +307,6 @@ func (i *instance) SearchTagsV2(ctx context.Context, req *tempopb.SearchTagsRequ
})
}
// add intrinsic tags if scope is none
if attributeScope == traceql.AttributeScopeNone {
resp.Scopes = append(resp.Scopes, &tempopb.SearchTagsV2Scope{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
})
}
return resp, nil
}

View File

@ -420,28 +420,13 @@ func cacheKeysForTestSearchTagValuesV2(tagKey, query string, limit int) []string
// TestInstanceSearchTagsSpecialCases tess that SearchTags errors on an unknown scope and
// returns known instrinics for the "intrinsic" scope
func TestInstanceSearchTagsSpecialCases(t *testing.T) {
func TestInstanceSearchUnknownScope(t *testing.T) {
i, _ := defaultInstance(t)
userCtx := user.InjectOrgID(context.Background(), "fake")
resp, err := i.SearchTags(userCtx, "foo")
require.Error(t, err)
require.Nil(t, resp)
resp, err = i.SearchTags(userCtx, "intrinsic")
require.NoError(t, err)
require.Equal(
t,
[]string{
"duration", "event:name", "event:timeSinceStart",
"instrumentation:name", "instrumentation:version",
"kind", "name", "rootName", "rootServiceName",
"span:duration", "span:kind", "span:name",
"span:status", "span:statusMessage", "status", "statusMessage",
"trace:duration", "trace:rootName", "trace:rootService", "traceDuration",
},
resp.TagNames,
)
}
// TestInstanceSearchMaxBytesPerTagValuesQueryReturnsPartial confirms that SearchTagValues returns

View File

@ -485,11 +485,6 @@ func (q *Querier) SearchTagsBlocks(ctx context.Context, req *tempopb.SearchTagsB
// flatten v2 response
for _, s := range v2Response.Scopes {
// SearchTags does not include intrinsics on an empty scope, but v2 does.
if req.SearchReq.Scope == "" && s.Name == api.ParamScopeIntrinsic {
continue
}
for _, t := range s.Tags {
distinctValues.Collect(t)
if distinctValues.Exceeded() {
@ -876,19 +871,10 @@ func (q *Querier) SearchBlock(ctx context.Context, req *tempopb.SearchBlockReque
}
func (q *Querier) internalTagsSearchBlockV2(ctx context.Context, req *tempopb.SearchTagsBlockRequest) (*tempopb.SearchTagsV2Response, error) {
// check if it's the special intrinsic scope
// note that every block search passes the same values up. this could be handled in the frontend and be far more efficient
// For the intrinsic scope there is nothing to do in the querier,
// these are always added by the frontend.
if req.SearchReq.Scope == api.ParamScopeIntrinsic {
return &tempopb.SearchTagsV2Response{
Scopes: []*tempopb.SearchTagsV2Scope{
{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
},
},
// no bytes were scanned to return the intrinsic values
Metrics: &tempopb.MetadataMetrics{InspectedBytes: 0},
}, nil
return &tempopb.SearchTagsV2Response{}, nil
}
tenantID, err := user.ExtractOrgID(ctx)
@ -930,20 +916,7 @@ func (q *Querier) internalTagsSearchBlockV2(ctx context.Context, req *tempopb.Se
query := traceql.ExtractMatchers(req.SearchReq.Query)
if traceql.IsEmptyQuery(query) {
resp, err := q.store.SearchTags(ctx, meta, req, opts)
if err != nil {
return nil, err
}
// add intrinsic tags if scope is none
if req.SearchReq.Scope == "" {
resp.Scopes = append(resp.Scopes, &tempopb.SearchTagsV2Scope{
Name: api.ParamScopeIntrinsic,
Tags: search.GetVirtualIntrinsicValues(),
})
}
return resp, nil
return q.store.SearchTags(ctx, meta, req, opts)
}
valueCollector := collector.NewScopedDistinctString(q.limits.MaxBytesPerTagValuesQuery(tenantID), req.MaxTagsPerScope, req.StaleValueThreshold)