diff --git a/CHANGELOG.md b/CHANGELOG.md index 93f25c6f936..c2602c30e5b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,7 +1,7 @@ ## main / unreleased * [FEATURE] TraceQL support for event scope and event:name intrinsic [#3708](https://github.com/grafana/tempo/pull/3708) (@stoewer) -* [FEATURE] Flush blocks to storage from the metrics-generator [#3628](https://github.com/grafana/tempo/pull/3628) [#3691](https://github.com/grafana/tempo/pull/3691) (@mapno) +* [FEATURE] Flush and query RF1 blocks for TraceQL metric queries [#3628](https://github.com/grafana/tempo/pull/3628) [#3691](https://github.com/grafana/tempo/pull/3691) [#3723](https://github.com/grafana/tempo/pull/3723) (@mapno) * [ENHANCEMENT] Tag value lookup use protobuf internally for improved latency [#3731](https://github.com/grafana/tempo/pull/3731) (@mdisibio) * [ENHANCEMENT] Improve use of OTEL semantic conventions on the service graph [#3711](https://github.com/grafana/tempo/pull/3711) (@zalegrala) * [ENHANCEMENT] Performance improvement for `rate() by ()` queries [#3719](https://github.com/grafana/tempo/pull/3719) (@mapno) diff --git a/cmd/tempo/app/modules.go b/cmd/tempo/app/modules.go index 84b33f3e8c3..391aa641559 100644 --- a/cmd/tempo/app/modules.go +++ b/cmd/tempo/app/modules.go @@ -263,9 +263,8 @@ func (t *App) initIngester() (services.Service, error) { } func (t *App) initGenerator() (services.Service, error) { - if t.cfg.Target == MetricsGenerator && - t.cfg.StorageConfig.Trace.Backend != "" && - t.cfg.Generator.Processor.LocalBlocks.FlushToStorage { + if t.cfg.Generator.Processor.LocalBlocks.FlushToStorage && + t.store == nil { return nil, fmt.Errorf("generator.processor.local-blocks.flush-to-storage is enabled but no storage backend is configured") } diff --git a/docs/sources/tempo/configuration/_index.md b/docs/sources/tempo/configuration/_index.md index eaf1bf2476e..3f4da295f52 100644 --- a/docs/sources/tempo/configuration/_index.md +++ b/docs/sources/tempo/configuration/_index.md @@ -553,6 +553,9 @@ query_frontend: # If set to a non-zero value, it's value will be used to decide if query is within SLO or not. # Query is within SLO if it returned 200 within duration_slo seconds OR processed throughput_slo bytes/s data. [throughput_bytes_slo: | default = 0 ] + + # If set to true, TraceQL metric queries will use RF1 blocks built and flushed by the metrics-generator. + [rf1_read_path: | default = false] ``` ## Querier diff --git a/docs/sources/tempo/configuration/manifest.md b/docs/sources/tempo/configuration/manifest.md index eac4a9bf84b..7bd468ba5db 100644 --- a/docs/sources/tempo/configuration/manifest.md +++ b/docs/sources/tempo/configuration/manifest.md @@ -74,6 +74,7 @@ server: log_format: logfmt log_level: info log_source_ips_enabled: false + log_source_ips_full: false log_source_ips_header: "" log_source_ips_regex: "" log_request_headers: false @@ -129,6 +130,7 @@ internal_server: log_format: logfmt log_level: info log_source_ips_enabled: false + log_source_ips_full: false log_source_ips_header: "" log_source_ips_regex: "" log_request_headers: false @@ -254,6 +256,9 @@ querier: external_endpoints: [] trace_by_id: query_timeout: 10s + metrics: + concurrent_blocks: 2 + time_overlap_cutoff: 0.2 max_concurrent_queries: 20 frontend_worker: frontend_address: 127.0.0.1:9095 @@ -285,11 +290,14 @@ querier: connect_timeout: 0s connect_backoff_base_delay: 0s connect_backoff_max_delay: 0s + shuffle_sharding_ingesters_enabled: false + shuffle_sharding_ingesters_lookback_period: 1h0m0s query_relevant_ingesters: false query_frontend: max_outstanding_per_tenant: 2000 querier_forget_delay: 0s max_batch_size: 5 + log_query_request_headers: "" max_retries: 2 search: concurrent_jobs: 1000 @@ -299,13 +307,14 @@ query_frontend: max_duration: 168h0m0s query_backend_after: 15m0s query_ingesters_until: 30m0s + ingester_shards: 1 trace_by_id: query_shards: 50 metrics: concurrent_jobs: 1000 target_bytes_per_job: 104857600 - max_duration: 0s - query_backend_after: 1h0m0s + max_duration: 3h0m0s + query_backend_after: 30m0s interval: 5m0s multi_tenant_queries_enabled: true compactor: @@ -410,7 +419,6 @@ ingester: min_ready_duration: 15s interface_names: - en0 - - bridge100 enable_inet6: false final_sleep: 0s tokens_file_path: "" @@ -492,6 +500,7 @@ metrics_generator: - db.name - db.system span_multiplier_key: "" + enable_virtual_node_label: false span_metrics: histogram_buckets: - 0.002 @@ -550,8 +559,10 @@ metrics_generator: max_block_bytes: 500000000 complete_block_timeout: 1h0m0s max_live_traces: 0 - concurrent_blocks: 10 filter_server_spans: true + flush_to_storage: false + concurrent_blocks: 10 + time_overlap_cutoff: 0.2 registry: collection_interval: 15s stale_duration: 15m0s @@ -620,6 +631,8 @@ storage: blocklist_poll_stale_tenant_index: 0s blocklist_poll_jitter_ms: 0 blocklist_poll_tolerate_consecutive_errors: 1 + empty_tenant_deletion_enabled: false + empty_tenant_deletion_age: 0s backend: local local: path: /var/tempo/traces diff --git a/modules/frontend/config.go b/modules/frontend/config.go index 17d80f15fa4..9903e87b4f9 100644 --- a/modules/frontend/config.go +++ b/modules/frontend/config.go @@ -85,6 +85,7 @@ func (cfg *Config) RegisterFlagsAndApplyDefaults(string, *flag.FlagSet) { ConcurrentRequests: defaultConcurrentRequests, TargetBytesPerRequest: defaultTargetBytesPerRequest, Interval: 5 * time.Minute, + RF1ReadPath: false, }, SLO: slo, } diff --git a/modules/frontend/metrics_query_range_sharder.go b/modules/frontend/metrics_query_range_sharder.go index 1551134a43a..96a66963b9f 100644 --- a/modules/frontend/metrics_query_range_sharder.go +++ b/modules/frontend/metrics_query_range_sharder.go @@ -25,11 +25,12 @@ import ( ) type queryRangeSharder struct { - next pipeline.AsyncRoundTripper[combiner.PipelineResponse] - reader tempodb.Reader - overrides overrides.Interface - cfg QueryRangeSharderConfig - logger log.Logger + next pipeline.AsyncRoundTripper[combiner.PipelineResponse] + reader tempodb.Reader + overrides overrides.Interface + cfg QueryRangeSharderConfig + logger log.Logger + replicationFactor uint32 } type QueryRangeSharderConfig struct { @@ -38,10 +39,15 @@ type QueryRangeSharderConfig struct { MaxDuration time.Duration `yaml:"max_duration"` QueryBackendAfter time.Duration `yaml:"query_backend_after,omitempty"` Interval time.Duration `yaml:"interval,omitempty"` + RF1ReadPath bool `yaml:"rf1_read_path,omitempty"` } // newAsyncQueryRangeSharder creates a sharding middleware for search func newAsyncQueryRangeSharder(reader tempodb.Reader, o overrides.Interface, cfg QueryRangeSharderConfig, logger log.Logger) pipeline.AsyncMiddleware[combiner.PipelineResponse] { + var replicationFactor uint32 + if cfg.RF1ReadPath { + replicationFactor = 1 + } return pipeline.AsyncMiddlewareFunc[combiner.PipelineResponse](func(next pipeline.AsyncRoundTripper[combiner.PipelineResponse]) pipeline.AsyncRoundTripper[combiner.PipelineResponse] { return queryRangeSharder{ next: next, @@ -50,6 +56,8 @@ func newAsyncQueryRangeSharder(reader tempodb.Reader, o overrides.Interface, cfg cfg: cfg, logger: logger, + + replicationFactor: replicationFactor, } }) } @@ -110,7 +118,19 @@ func (s queryRangeSharder) RoundTrip(r *http.Request) (pipeline.Responses[combin reqCh <- generatorReq } - totalJobs, totalBlocks, totalBlockBytes := s.backendRequests(ctx, tenantID, r, *req, now, samplingRate, targetBytesPerRequest, interval, reqCh, nil) + var ( + totalJobs, totalBlocks uint32 + totalBlockBytes uint64 + ) + if s.cfg.RF1ReadPath { + totalJobs, totalBlocks, totalBlockBytes = s.backendRequests(ctx, tenantID, r, *req, now, samplingRate, targetBytesPerRequest, interval, reqCh) + } else { + totalJobs, totalBlocks, totalBlockBytes = s.shardedBackendRequests(ctx, tenantID, r, *req, now, samplingRate, targetBytesPerRequest, interval, reqCh, nil) + } + + span.SetTag("totalJobs", totalJobs) + span.SetTag("totalBlocks", totalBlocks) + span.SetTag("totalBlockBytes", totalBlockBytes) // send a job to communicate the search metrics. this is consumed by the combiner to calculate totalblocks/bytes/jobs var jobMetricsResponse pipeline.Responses[combiner.PipelineResponse] @@ -142,7 +162,8 @@ func (s *queryRangeSharder) blockMetas(start, end int64, tenantID string) []*bac metas := make([]*backend.BlockMeta, 0, len(allMetas)/50) // divide by 50 for luck for _, m := range allMetas { if m.StartTime.UnixNano() <= end && - m.EndTime.UnixNano() >= start { + m.EndTime.UnixNano() >= start && + m.ReplicationFactor == s.replicationFactor { metas = append(metas, m) } } @@ -150,7 +171,7 @@ func (s *queryRangeSharder) blockMetas(start, end int64, tenantID string) []*bac return metas } -func (s *queryRangeSharder) backendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, now time.Time, samplingRate float64, targetBytesPerRequest int, interval time.Duration, reqCh chan *http.Request, _ func(error)) (totalJobs, totalBlocks uint32, totalBlockBytes uint64) { +func (s *queryRangeSharder) shardedBackendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, now time.Time, samplingRate float64, targetBytesPerRequest int, interval time.Duration, reqCh chan *http.Request, _ func(error)) (totalJobs, totalBlocks uint32, totalBlockBytes uint64) { // request without start or end, search only in generator if searchReq.Start == 0 || searchReq.End == 0 { close(reqCh) @@ -218,13 +239,13 @@ func (s *queryRangeSharder) backendRequests(ctx context.Context, tenantID string } go func() { - s.buildBackendRequests(ctx, tenantID, parent, backendReq, samplingRate, targetBytesPerRequest, interval, reqCh, nil) + s.buildShardedBackendRequests(ctx, tenantID, parent, backendReq, samplingRate, targetBytesPerRequest, interval, reqCh) }() return } -func (s *queryRangeSharder) buildBackendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, samplingRate float64, targetBytesPerRequest int, interval time.Duration, reqCh chan *http.Request, _ func(error)) { +func (s *queryRangeSharder) buildShardedBackendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, samplingRate float64, targetBytesPerRequest int, interval time.Duration, reqCh chan *http.Request) { defer close(reqCh) var ( @@ -260,7 +281,6 @@ func (s *queryRangeSharder) buildBackendRequests(ctx context.Context, tenantID s shardR.End = thisEnd shardR.ShardID = i shardR.ShardCount = shards - httpReq := s.toUpstreamRequest(ctx, shardR, parent, tenantID) if samplingRate != 1.0 { shardR.ShardID *= uint32(1.0 / samplingRate) @@ -283,6 +303,105 @@ func (s *queryRangeSharder) buildBackendRequests(ctx context.Context, tenantID s } } +func (s *queryRangeSharder) backendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, now time.Time, _ float64, targetBytesPerRequest int, _ time.Duration, reqCh chan *http.Request) (totalJobs, totalBlocks uint32, totalBlockBytes uint64) { + // request without start or end, search only in generator + if searchReq.Start == 0 || searchReq.End == 0 { + close(reqCh) + return + } + + // Make a copy and limit to backend time range. + backendReq := searchReq + backendReq.Start, backendReq.End = s.backendRange(now, backendReq.Start, backendReq.End, s.cfg.QueryBackendAfter) + alignTimeRange(&backendReq) + + // If empty window then no need to search backend + if backendReq.Start == backendReq.End { + close(reqCh) + return + } + + // Blocks within overall time range. This is just for instrumentation, more precise time + // range is checked for each window. + blocks := s.blockMetas(int64(backendReq.Start), int64(backendReq.End), tenantID) + if len(blocks) == 0 { + // no need to search backend + close(reqCh) + return + } + + // calculate metrics to return to the caller + totalBlocks = uint32(len(blocks)) + for _, b := range blocks { + p := pagesPerRequest(b, targetBytesPerRequest) + + totalJobs += b.TotalRecords / uint32(p) + if int(b.TotalRecords)%p != 0 { + totalJobs++ + } + totalBlockBytes += b.Size + } + + go func() { + s.buildBackendRequests(ctx, tenantID, parent, backendReq, blocks, targetBytesPerRequest, reqCh) + }() + + return +} + +func (s *queryRangeSharder) buildBackendRequests(ctx context.Context, tenantID string, parent *http.Request, searchReq tempopb.QueryRangeRequest, metas []*backend.BlockMeta, targetBytesPerRequest int, reqCh chan<- *http.Request) { + defer close(reqCh) + + for _, m := range metas { + pages := pagesPerRequest(m, targetBytesPerRequest) + if pages == 0 { + continue + } + + for startPage := 0; startPage < int(m.TotalRecords); startPage += pages { + subR := parent.Clone(ctx) + + dc, err := m.DedicatedColumns.ToTempopb() + if err != nil { + // errFn(fmt.Errorf("failed to convert dedicated columns. block: %s tempopb: %w", blockID, err)) + continue + } + + queryRangeReq := &tempopb.QueryRangeRequest{ + Query: searchReq.Query, + Start: max(searchReq.Start, uint64(m.StartTime.UnixNano())), + End: min(searchReq.End, uint64(m.EndTime.UnixNano())), + Step: searchReq.Step, + // ShardID: uint32, // No sharding with RF=1 + // ShardCount: uint32, // No sharding with RF=1 + QueryMode: searchReq.QueryMode, + // New RF1 fields + BlockID: m.BlockID.String(), + StartPage: uint32(startPage), + PagesToSearch: uint32(pages), + Version: m.Version, + Encoding: m.Encoding.String(), + Size_: m.Size, + FooterSize: m.FooterSize, + DedicatedColumns: dc, + } + alignTimeRange(queryRangeReq) + queryRangeReq.End += queryRangeReq.Step + + subR = api.BuildQueryRangeRequest(subR, queryRangeReq) + + prepareRequestForQueriers(subR, tenantID, subR.URL.Path, subR.URL.Query()) + // TODO: Handle sampling rate + + select { + case reqCh <- subR: + case <-ctx.Done(): + return + } + } + } +} + func (s *queryRangeSharder) backendRange(now time.Time, start, end uint64, queryBackendAfter time.Duration) (uint64, uint64) { backendAfter := uint64(now.Add(-queryBackendAfter).UnixNano()) @@ -319,13 +438,6 @@ func (s *queryRangeSharder) generatorRequest(searchReq tempopb.QueryRangeRequest searchReq.QueryMode = querier.QueryModeRecent - // No sharding on the generators (unnecessary), but we do apply sampling - // rates. In this case we execute a single arbitrary shard. Choosing - // the last shard works. The first shard should be avoided because it is - // weighted slightly off due to int63/128 sharding boundaries. - searchReq.ShardID = uint32(1.0 / samplingRate) - searchReq.ShardCount = uint32(1.0 / samplingRate) - return s.toUpstreamRequest(parent.Context(), searchReq, parent, tenantID) } diff --git a/modules/frontend/search_sharder.go b/modules/frontend/search_sharder.go index c4a9734fc17..f1991a7a067 100644 --- a/modules/frontend/search_sharder.go +++ b/modules/frontend/search_sharder.go @@ -38,11 +38,6 @@ type SearchSharderConfig struct { IngesterShards int `yaml:"ingester_shards,omitempty"` } -type backendReqMsg struct { - req *http.Request - err error -} - type asyncSearchSharder struct { next pipeline.AsyncRoundTripper[combiner.PipelineResponse] reader tempodb.Reader @@ -147,7 +142,8 @@ func (s *asyncSearchSharder) blockMetas(start, end int64, tenantID string) []*ba metas := make([]*backend.BlockMeta, 0, len(allMetas)/50) // divide by 50 for luck for _, m := range allMetas { if m.StartTime.Unix() <= end && - m.EndTime.Unix() >= start { + m.EndTime.Unix() >= start && + m.ReplicationFactor == backend.DefaultReplicationFactor { // This check skips generator blocks (RF=1) metas = append(metas, m) } } diff --git a/modules/generator/processor/localblocks/processor.go b/modules/generator/processor/localblocks/processor.go index 59b671fc325..8666a632d01 100644 --- a/modules/generator/processor/localblocks/processor.go +++ b/modules/generator/processor/localblocks/processor.go @@ -715,7 +715,7 @@ func (p *Processor) resetHeadBlock() error { BlockID: uuid.New(), TenantID: p.tenant, DedicatedColumns: p.overrides.DedicatedColumns(p.tenant), - ReplicationFactor: 1, + ReplicationFactor: backend.MetricsGeneratorReplicationFactor, } block, err := p.wal.NewBlock(meta, model.CurrentEncoding) if err != nil { diff --git a/modules/querier/querier.go b/modules/querier/querier.go index a12e1f60cb7..cbdc5a4b041 100644 --- a/modules/querier/querier.go +++ b/modules/querier/querier.go @@ -284,6 +284,7 @@ func (q *Querier) FindTraceByID(ctx context.Context, req *tempopb.TraceByIDReque span.LogFields(ot_log.String("timeEnd", fmt.Sprint(timeEnd))) opts := common.DefaultSearchOptionsWithMaxBytes(maxBytes) + opts.BlockReplicationFactor = backend.DefaultReplicationFactor partialTraces, blockErrs, err := q.store.Find(ctx, userID, req.TraceID, req.BlockStart, req.BlockEnd, timeStart, timeEnd, opts) if err != nil { retErr := fmt.Errorf("error querying store in Querier.FindTraceByID: %w", err) diff --git a/modules/querier/querier_query_range.go b/modules/querier/querier_query_range.go index 43308c43859..cd1b21e3535 100644 --- a/modules/querier/querier_query_range.go +++ b/modules/querier/querier_query_range.go @@ -6,6 +6,7 @@ import ( "time" "github.com/go-kit/log/level" + "github.com/google/uuid" "github.com/grafana/dskit/ring" "github.com/grafana/dskit/user" "github.com/grafana/tempo/pkg/boundedwaitgroup" @@ -24,6 +25,10 @@ func (q *Querier) QueryRange(ctx context.Context, req *tempopb.QueryRangeRequest return q.queryRangeRecent(ctx, req) } + if req.BlockID != "" { // RF1 search + return q.queryBlock(ctx, req) + } + // Backend requests go here return q.queryBackend(ctx, req) } @@ -59,6 +64,84 @@ func (q *Querier) queryRangeRecent(ctx context.Context, req *tempopb.QueryRangeR return c.Response(), nil } +func (q *Querier) queryBlock(ctx context.Context, req *tempopb.QueryRangeRequest) (*tempopb.QueryRangeResponse, error) { + tenantID, err := user.ExtractOrgID(ctx) + if err != nil { + return nil, fmt.Errorf("error extracting org id in Querier.BackendSearch: %w", err) + } + + blockID, err := uuid.Parse(req.BlockID) + if err != nil { + return nil, err + } + + enc, err := backend.ParseEncoding(req.Encoding) + if err != nil { + return nil, err + } + + dc, err := backend.DedicatedColumnsFromTempopb(req.DedicatedColumns) + if err != nil { + return nil, err + } + + meta := &backend.BlockMeta{ + Version: req.Version, + TenantID: tenantID, + StartTime: time.Unix(0, int64(req.Start)), + EndTime: time.Unix(0, int64(req.End)), + Encoding: enc, + // IndexPageSize: req.IndexPageSize, + // TotalRecords: req.TotalRecords, + BlockID: blockID, + // DataEncoding: req.DataEncoding, + Size: req.Size_, + FooterSize: req.FooterSize, + DedicatedColumns: dc, + } + + opts := common.DefaultSearchOptions() + opts.StartPage = int(req.StartPage) + opts.TotalPages = int(req.PagesToSearch) + + unsafe := q.limits.UnsafeQueryHints(tenantID) + + expr, err := traceql.Parse(req.Query) + if err != nil { + return nil, err + } + + timeOverlapCutoff := q.cfg.Metrics.TimeOverlapCutoff + if v, ok := expr.Hints.GetFloat(traceql.HintTimeOverlapCutoff, unsafe); ok && v >= 0 && v <= 1.0 { + timeOverlapCutoff = v + } + + eval, err := traceql.NewEngine().CompileMetricsQueryRange(req, false, timeOverlapCutoff, unsafe) + if err != nil { + return nil, err + } + + f := traceql.NewSpansetFetcherWrapper(func(ctx context.Context, req traceql.FetchSpansRequest) (traceql.FetchSpansResponse, error) { + return q.store.Fetch(ctx, meta, req, opts) + }) + err = eval.Do(ctx, f, uint64(meta.StartTime.UnixNano()), uint64(meta.EndTime.UnixNano())) + if err != nil { + return nil, err + } + + res := eval.Results() + + inspectedBytes, spansTotal, _ := eval.Metrics() + + return &tempopb.QueryRangeResponse{ + Series: queryRangeTraceQLToProto(res, req), + Metrics: &tempopb.SearchMetrics{ + InspectedBytes: inspectedBytes, + InspectedSpans: spansTotal, + }, + }, nil +} + func (q *Querier) queryBackend(ctx context.Context, req *tempopb.QueryRangeRequest) (*tempopb.QueryRangeResponse, error) { ctx, cancel := context.WithCancel(ctx) defer cancel() diff --git a/pkg/api/http.go b/pkg/api/http.go index ea09dba33cb..88ae4244806 100644 --- a/pkg/api/http.go +++ b/pkg/api/http.go @@ -353,14 +353,54 @@ func ParseQueryRangeRequest(r *http.Request) (*tempopb.QueryRangeRequest, error) req.Step = uint64(step.Nanoseconds()) shardCount, _ := extractQueryParam(r, urlParamShardCount) - if of, err := strconv.Atoi(shardCount); err == nil { - req.ShardCount = uint32(of) + if shardCount, err := strconv.Atoi(shardCount); err == nil { + req.ShardCount = uint32(shardCount) } shard, _ := extractQueryParam(r, urlParamShard) if shard, err := strconv.Atoi(shard); err == nil { req.ShardID = uint32(shard) } + // New RF1 params + blockID, _ := extractQueryParam(r, urlParamBlockID) + if blockID, err := uuid.Parse(blockID); err == nil { + req.BlockID = blockID.String() + } + + startPage, _ := extractQueryParam(r, urlParamStartPage) + if startPage, err := strconv.Atoi(startPage); err == nil { + req.StartPage = uint32(startPage) + } + + pagesToSearch, _ := extractQueryParam(r, urlParamPagesToSearch) + if of, err := strconv.Atoi(pagesToSearch); err == nil { + req.PagesToSearch = uint32(of) + } + + version, _ := extractQueryParam(r, urlParamVersion) + req.Version = version + + encoding, _ := extractQueryParam(r, urlParamEncoding) + req.Encoding = encoding + + size, _ := extractQueryParam(r, urlParamSize) + if size, err := strconv.Atoi(size); err == nil { + req.Size_ = uint64(size) + } + + footerSize, _ := extractQueryParam(r, urlParamFooterSize) + if footerSize, err := strconv.Atoi(footerSize); err == nil { + req.FooterSize = uint32(footerSize) + } + + dedicatedColumns, _ := extractQueryParam(r, urlParamDedicatedColumns) + if len(dedicatedColumns) > 0 { + err := json.Unmarshal([]byte(dedicatedColumns), &req.DedicatedColumns) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, fmt.Errorf("failed to parse dedicated columns: %w", err).Error()) + } + } + return req, nil } @@ -382,10 +422,23 @@ func BuildQueryRangeRequest(req *http.Request, searchReq *tempopb.QueryRangeRequ q.Set(urlParamShard, strconv.FormatUint(uint64(searchReq.ShardID), 10)) q.Set(urlParamShardCount, strconv.FormatUint(uint64(searchReq.ShardCount), 10)) q.Set(QueryModeKey, searchReq.QueryMode) + // New RF1 params + q.Set(urlParamBlockID, searchReq.BlockID) + q.Set(urlParamStartPage, strconv.Itoa(int(searchReq.StartPage))) + q.Set(urlParamPagesToSearch, strconv.Itoa(int(searchReq.PagesToSearch))) + q.Set(urlParamVersion, searchReq.Version) + q.Set(urlParamEncoding, searchReq.Encoding) + q.Set(urlParamSize, strconv.Itoa(int(searchReq.Size_))) + q.Set(urlParamFooterSize, strconv.Itoa(int(searchReq.FooterSize))) + if len(searchReq.DedicatedColumns) > 0 { + columnsJSON, _ := json.Marshal(searchReq.DedicatedColumns) + q.Set(urlParamDedicatedColumns, string(columnsJSON)) + } if len(searchReq.Query) > 0 { q.Set(urlParamQuery, searchReq.Query) } + req.URL.RawQuery = q.Encode() return req diff --git a/pkg/tempopb/tempo.pb.go b/pkg/tempopb/tempo.pb.go index aa0115e25f3..f4a2e4c10a2 100644 --- a/pkg/tempopb/tempo.pb.go +++ b/pkg/tempopb/tempo.pb.go @@ -2749,6 +2749,15 @@ type QueryRangeRequest struct { ShardID uint32 `protobuf:"varint,5,opt,name=shardID,proto3" json:"shardID,omitempty"` ShardCount uint32 `protobuf:"varint,6,opt,name=shardCount,proto3" json:"shardCount,omitempty"` QueryMode string `protobuf:"bytes,7,opt,name=queryMode,proto3" json:"queryMode,omitempty"` + // New RF1 fields + BlockID string `protobuf:"bytes,8,opt,name=blockID,proto3" json:"blockID,omitempty"` + StartPage uint32 `protobuf:"varint,9,opt,name=startPage,proto3" json:"startPage,omitempty"` + PagesToSearch uint32 `protobuf:"varint,10,opt,name=pagesToSearch,proto3" json:"pagesToSearch,omitempty"` + Version string `protobuf:"bytes,11,opt,name=version,proto3" json:"version,omitempty"` + Encoding string `protobuf:"bytes,12,opt,name=encoding,proto3" json:"encoding,omitempty"` + Size_ uint64 `protobuf:"varint,13,opt,name=size,proto3" json:"size,omitempty"` + FooterSize uint32 `protobuf:"varint,14,opt,name=footerSize,proto3" json:"footerSize,omitempty"` + DedicatedColumns []*DedicatedColumn `protobuf:"bytes,15,rep,name=dedicatedColumns,proto3" json:"dedicatedColumns,omitempty"` } func (m *QueryRangeRequest) Reset() { *m = QueryRangeRequest{} } @@ -2833,6 +2842,62 @@ func (m *QueryRangeRequest) GetQueryMode() string { return "" } +func (m *QueryRangeRequest) GetBlockID() string { + if m != nil { + return m.BlockID + } + return "" +} + +func (m *QueryRangeRequest) GetStartPage() uint32 { + if m != nil { + return m.StartPage + } + return 0 +} + +func (m *QueryRangeRequest) GetPagesToSearch() uint32 { + if m != nil { + return m.PagesToSearch + } + return 0 +} + +func (m *QueryRangeRequest) GetVersion() string { + if m != nil { + return m.Version + } + return "" +} + +func (m *QueryRangeRequest) GetEncoding() string { + if m != nil { + return m.Encoding + } + return "" +} + +func (m *QueryRangeRequest) GetSize_() uint64 { + if m != nil { + return m.Size_ + } + return 0 +} + +func (m *QueryRangeRequest) GetFooterSize() uint32 { + if m != nil { + return m.FooterSize + } + return 0 +} + +func (m *QueryRangeRequest) GetDedicatedColumns() []*DedicatedColumn { + if m != nil { + return m.DedicatedColumns + } + return nil +} + type QueryRangeResponse struct { Series []*TimeSeries `protobuf:"bytes,1,rep,name=series,proto3" json:"series,omitempty"` Metrics *SearchMetrics `protobuf:"bytes,2,opt,name=metrics,proto3" json:"metrics,omitempty"` @@ -3057,170 +3122,173 @@ func init() { func init() { proto.RegisterFile("pkg/tempopb/tempo.proto", fileDescriptor_f22805646f4f62b6) } var fileDescriptor_f22805646f4f62b6 = []byte{ - // 2595 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x1a, 0xcb, 0x6e, 0x23, 0xc7, - 0x51, 0x23, 0xbe, 0x8b, 0x94, 0x44, 0xf5, 0xae, 0x65, 0x2e, 0x65, 0x4b, 0xca, 0x78, 0xe1, 0x28, - 0x7e, 0x50, 0x5a, 0x7a, 0x17, 0xf6, 0xda, 0x89, 0x03, 0x69, 0xa5, 0xc8, 0xb2, 0xf5, 0x72, 0x93, - 0x96, 0x8d, 0xc0, 0x80, 0x30, 0x24, 0x7b, 0xb9, 0x03, 0x91, 0x33, 0xf4, 0xcc, 0x50, 0x59, 0xe6, - 0x18, 0x20, 0x01, 0x02, 0xe4, 0x90, 0x43, 0x72, 0xf0, 0x31, 0xa7, 0x20, 0xe7, 0x9c, 0x73, 0x0a, - 0x10, 0xf8, 0x12, 0xc3, 0x40, 0x2e, 0x46, 0x0e, 0x46, 0x60, 0x1f, 0xf2, 0x01, 0xf9, 0x81, 0xa0, - 0xaa, 0xbb, 0xe7, 0x41, 0x52, 0xda, 0x6c, 0xb2, 0x46, 0x7c, 0xf0, 0x49, 0x5d, 0xd5, 0x35, 0xd5, - 0xd5, 0xf5, 0xae, 0xa6, 0xe0, 0xe9, 0xc1, 0x79, 0x77, 0x23, 0x10, 0xfd, 0x81, 0x3b, 0x68, 0xc9, - 0xbf, 0xb5, 0x81, 0xe7, 0x06, 0x2e, 0xcb, 0x29, 0x64, 0x75, 0xa9, 0xed, 0xf6, 0xfb, 0xae, 0xb3, - 0x71, 0x71, 0x6b, 0x43, 0xae, 0x24, 0x41, 0xf5, 0xe5, 0xae, 0x1d, 0x3c, 0x18, 0xb6, 0x6a, 0x6d, - 0xb7, 0xbf, 0xd1, 0x75, 0xbb, 0xee, 0x06, 0xa1, 0x5b, 0xc3, 0xfb, 0x04, 0x11, 0x40, 0x2b, 0x45, - 0x7e, 0x3d, 0xf0, 0xac, 0xb6, 0x40, 0x2e, 0xb4, 0x90, 0x58, 0xf3, 0x17, 0x06, 0x94, 0x9b, 0x08, - 0x6f, 0x8f, 0xf6, 0x77, 0xb8, 0xf8, 0x68, 0x28, 0xfc, 0x80, 0x55, 0x20, 0x47, 0x34, 0xfb, 0x3b, - 0x15, 0x63, 0xcd, 0x58, 0x2f, 0x71, 0x0d, 0xb2, 0x15, 0x80, 0x56, 0xcf, 0x6d, 0x9f, 0x37, 0x02, - 0xcb, 0x0b, 0x2a, 0xb3, 0x6b, 0xc6, 0x7a, 0x81, 0xc7, 0x30, 0xac, 0x0a, 0x79, 0x82, 0x76, 0x9d, - 0x4e, 0x25, 0x45, 0xbb, 0x21, 0xcc, 0x9e, 0x81, 0xc2, 0x47, 0x43, 0xe1, 0x8d, 0x0e, 0xdd, 0x8e, - 0xa8, 0x64, 0x68, 0x33, 0x42, 0x98, 0x0e, 0x2c, 0xc6, 0xe4, 0xf0, 0x07, 0xae, 0xe3, 0x0b, 0x76, - 0x13, 0x32, 0x74, 0x32, 0x89, 0x51, 0xac, 0xcf, 0xd7, 0x94, 0x4e, 0x6a, 0x44, 0xca, 0xe5, 0x26, - 0x7b, 0x05, 0x72, 0x7d, 0x11, 0x78, 0x76, 0xdb, 0x27, 0x89, 0x8a, 0xf5, 0x1b, 0x49, 0x3a, 0x64, - 0x79, 0x28, 0x09, 0xb8, 0xa6, 0x34, 0x59, 0xec, 0xde, 0x6a, 0xd3, 0xfc, 0x74, 0x16, 0xe6, 0x1a, - 0xc2, 0xf2, 0xda, 0x0f, 0xb4, 0x26, 0x5e, 0x87, 0x74, 0xd3, 0xea, 0xfa, 0x15, 0x63, 0x2d, 0xb5, - 0x5e, 0xac, 0xaf, 0x85, 0x7c, 0x13, 0x54, 0x35, 0x24, 0xd9, 0x75, 0x02, 0x6f, 0xb4, 0x9d, 0xfe, - 0xe4, 0x8b, 0xd5, 0x19, 0x4e, 0xdf, 0xb0, 0x9b, 0x30, 0x77, 0x68, 0x3b, 0x3b, 0x43, 0xcf, 0x0a, - 0x6c, 0xd7, 0x39, 0x94, 0xc2, 0xcd, 0xf1, 0x24, 0x92, 0xa8, 0xac, 0x87, 0x31, 0xaa, 0x94, 0xa2, - 0x8a, 0x23, 0xd9, 0x75, 0xc8, 0x1c, 0xd8, 0x7d, 0x3b, 0xa8, 0xa4, 0x69, 0x57, 0x02, 0x88, 0xf5, - 0xc9, 0x10, 0x19, 0x89, 0x25, 0x80, 0x95, 0x21, 0x25, 0x9c, 0x4e, 0x25, 0x4b, 0x38, 0x5c, 0x22, - 0xdd, 0xbb, 0xa8, 0xe8, 0x4a, 0x9e, 0xb4, 0x2e, 0x01, 0xb6, 0x0e, 0x0b, 0x8d, 0x81, 0xe5, 0xf8, - 0x27, 0xc2, 0xc3, 0xbf, 0x0d, 0x11, 0x54, 0x0a, 0xf4, 0xcd, 0x38, 0xba, 0xfa, 0x2a, 0x14, 0xc2, - 0x2b, 0x22, 0xfb, 0x73, 0x31, 0x22, 0x8b, 0x14, 0x38, 0x2e, 0x91, 0xfd, 0x85, 0xd5, 0x1b, 0x0a, - 0xe5, 0x0f, 0x12, 0x78, 0x7d, 0xf6, 0x35, 0xc3, 0xfc, 0x4b, 0x0a, 0x98, 0x54, 0xd5, 0x36, 0x7a, - 0x81, 0xd6, 0xea, 0x6d, 0x28, 0xf8, 0x5a, 0x81, 0xca, 0xb4, 0x4b, 0xd3, 0x55, 0xcb, 0x23, 0x42, - 0xf4, 0x4a, 0xf2, 0xa5, 0xfd, 0x1d, 0x75, 0x90, 0x06, 0xd1, 0xb3, 0xe8, 0xea, 0x27, 0x56, 0x57, - 0x28, 0xfd, 0x45, 0x08, 0xd4, 0xf0, 0xc0, 0xea, 0x0a, 0xbf, 0xe9, 0x4a, 0xd6, 0x4a, 0x87, 0x49, - 0x24, 0x7a, 0xae, 0x70, 0xda, 0x6e, 0xc7, 0x76, 0xba, 0xca, 0x39, 0x43, 0x18, 0x39, 0xd8, 0x4e, - 0x47, 0x3c, 0x44, 0x76, 0x0d, 0xfb, 0xa7, 0x42, 0xe9, 0x36, 0x89, 0x64, 0x26, 0x94, 0x02, 0x37, - 0xb0, 0x7a, 0x5c, 0xb4, 0x5d, 0xaf, 0xe3, 0x57, 0x72, 0x44, 0x94, 0xc0, 0x21, 0x4d, 0xc7, 0x0a, - 0xac, 0x5d, 0x7d, 0x92, 0x34, 0x48, 0x02, 0x87, 0xf7, 0xbc, 0x10, 0x9e, 0x6f, 0xbb, 0x0e, 0xd9, - 0xa3, 0xc0, 0x35, 0xc8, 0x18, 0xa4, 0x7d, 0x3c, 0x1e, 0xd6, 0x8c, 0xf5, 0x34, 0xa7, 0x35, 0x46, - 0xe4, 0x7d, 0xd7, 0x0d, 0x84, 0x47, 0x82, 0x15, 0xe9, 0xcc, 0x18, 0x86, 0xed, 0x40, 0xb9, 0x23, - 0x3a, 0x76, 0xdb, 0x0a, 0x44, 0xe7, 0x9e, 0xdb, 0x1b, 0xf6, 0x1d, 0xbf, 0x52, 0x22, 0x6f, 0xae, - 0x84, 0x2a, 0xdf, 0x49, 0x12, 0xf0, 0x89, 0x2f, 0xcc, 0x3f, 0x1b, 0xb0, 0x30, 0x46, 0xc5, 0x6e, - 0x43, 0xc6, 0x6f, 0xbb, 0x03, 0xa9, 0xf1, 0xf9, 0xfa, 0xca, 0x65, 0xec, 0x6a, 0x0d, 0xa4, 0xe2, - 0x92, 0x18, 0xef, 0xe0, 0x58, 0x7d, 0xed, 0x2b, 0xb4, 0x66, 0xb7, 0x20, 0x1d, 0x8c, 0x06, 0x32, - 0xca, 0xe7, 0xeb, 0xcf, 0x5e, 0xca, 0xa8, 0x39, 0x1a, 0x08, 0x4e, 0xa4, 0xe6, 0x2a, 0x64, 0x88, - 0x2d, 0xcb, 0x43, 0xba, 0x71, 0xb2, 0x75, 0x54, 0x9e, 0x61, 0x25, 0xc8, 0xf3, 0xdd, 0xc6, 0xf1, - 0x7b, 0xfc, 0xde, 0x6e, 0xd9, 0x30, 0x19, 0xa4, 0x91, 0x9c, 0x01, 0x64, 0x1b, 0x4d, 0xbe, 0x7f, - 0xb4, 0x57, 0x9e, 0x31, 0x1f, 0xc2, 0xbc, 0xf6, 0x2e, 0x95, 0x60, 0x6e, 0x43, 0x96, 0x72, 0x88, - 0x8e, 0xf0, 0x67, 0x92, 0x99, 0x43, 0x52, 0x1f, 0x8a, 0xc0, 0x42, 0x0b, 0x71, 0x45, 0xcb, 0x36, - 0xc7, 0x13, 0xce, 0xb8, 0xf7, 0x4e, 0x64, 0x9b, 0xbf, 0xa5, 0xe0, 0xda, 0x14, 0x8e, 0xe3, 0x99, - 0xb6, 0x10, 0x65, 0xda, 0x75, 0x58, 0xf0, 0x5c, 0x37, 0x68, 0x08, 0xef, 0xc2, 0x6e, 0x8b, 0xa3, - 0x48, 0x65, 0xe3, 0x68, 0xf4, 0x4e, 0x44, 0x11, 0x7b, 0xa2, 0x93, 0x89, 0x37, 0x89, 0x64, 0x2f, - 0xc1, 0x22, 0x85, 0x44, 0xd3, 0xee, 0x8b, 0xf7, 0x1c, 0xfb, 0xe1, 0x91, 0xe5, 0xb8, 0x14, 0x09, - 0x69, 0x3e, 0xb9, 0x81, 0x5e, 0xd5, 0x89, 0x52, 0x92, 0x4c, 0x2f, 0x31, 0x0c, 0x7b, 0x01, 0x72, - 0xbe, 0xca, 0x19, 0x59, 0xd2, 0x40, 0x39, 0xd2, 0x80, 0xc4, 0x73, 0x4d, 0xc0, 0x5e, 0x82, 0xbc, - 0x5a, 0x62, 0x4c, 0xa4, 0xa6, 0x12, 0x87, 0x14, 0x8c, 0x43, 0xc9, 0x97, 0x97, 0x6b, 0x04, 0x56, - 0xe0, 0x57, 0xf2, 0xf4, 0x45, 0xed, 0x2a, 0xbb, 0xd4, 0x1a, 0xb1, 0x0f, 0x28, 0x49, 0xf1, 0x04, - 0x8f, 0xea, 0x29, 0x2c, 0x4e, 0x90, 0x4c, 0xc9, 0x63, 0x2f, 0xc6, 0xf3, 0x58, 0xb1, 0xfe, 0x54, - 0xcc, 0xa8, 0xd1, 0xc7, 0xf1, 0xf4, 0x76, 0x00, 0xa5, 0xf8, 0x16, 0xe5, 0xa1, 0x81, 0xe5, 0xdc, - 0x73, 0x87, 0x4e, 0x40, 0x8c, 0x31, 0x0f, 0x69, 0x04, 0xea, 0x54, 0x78, 0x9e, 0xeb, 0xc9, 0x6d, - 0x59, 0x0c, 0x62, 0x18, 0xf3, 0xe7, 0x06, 0xe4, 0x94, 0x3e, 0xd8, 0x73, 0x90, 0xc1, 0x0f, 0xb5, - 0x5b, 0xce, 0x25, 0x14, 0xc6, 0xe5, 0x1e, 0x3a, 0x4f, 0xdf, 0x0a, 0xda, 0x0f, 0x44, 0x47, 0x71, - 0xd3, 0x20, 0x7b, 0x03, 0xc0, 0x0a, 0x02, 0xcf, 0x6e, 0x0d, 0x03, 0x81, 0x15, 0x05, 0x79, 0x2c, - 0x87, 0x3c, 0x54, 0x17, 0x71, 0x71, 0xab, 0xf6, 0x8e, 0x18, 0x9d, 0xe2, 0x6d, 0x78, 0x8c, 0x1c, - 0x63, 0x3d, 0x8d, 0xc7, 0xb0, 0x25, 0xc8, 0xe2, 0x41, 0xa1, 0x6f, 0x2a, 0x68, 0x6a, 0x08, 0x4f, - 0x75, 0xaf, 0xd4, 0x65, 0xee, 0x75, 0x13, 0xe6, 0xb4, 0x33, 0x21, 0xec, 0x2b, 0x47, 0x4c, 0x22, - 0xc7, 0x6e, 0x91, 0x79, 0xbc, 0x5b, 0x7c, 0x1c, 0xd6, 0x72, 0x15, 0x8c, 0x18, 0x51, 0xb6, 0xe3, - 0x0f, 0x44, 0x3b, 0x10, 0x9d, 0xa6, 0x0e, 0x7a, 0xaa, 0x77, 0x63, 0x68, 0xf6, 0x3c, 0xcc, 0x87, - 0xa8, 0xed, 0x11, 0x1e, 0x3e, 0x4b, 0xf2, 0x8d, 0x61, 0xd9, 0x1a, 0x14, 0x29, 0xbb, 0x53, 0x71, - 0xd3, 0x95, 0x3b, 0x8e, 0xc2, 0x8b, 0xb6, 0xdd, 0xfe, 0xa0, 0x27, 0x02, 0xd1, 0x79, 0xdb, 0x6d, - 0xf9, 0xba, 0xf6, 0x24, 0x90, 0xe8, 0x37, 0xf4, 0x11, 0x51, 0xc8, 0x60, 0x8b, 0x10, 0x28, 0x77, - 0xc4, 0x52, 0x8a, 0x93, 0x25, 0x71, 0xc6, 0xd1, 0x09, 0xb9, 0xa9, 0x86, 0x53, 0x0d, 0x8a, 0xcb, - 0x4d, 0x58, 0xf3, 0x5d, 0x8c, 0x07, 0x54, 0x0d, 0x56, 0x75, 0x5d, 0x94, 0xaf, 0xeb, 0x74, 0x2e, - 0x8d, 0xad, 0xd2, 0x75, 0xd8, 0x62, 0xa4, 0xa6, 0xb4, 0x18, 0xe9, 0xb0, 0xc5, 0x30, 0x3f, 0x4d, - 0xc1, 0x52, 0xc4, 0x33, 0x51, 0xed, 0x5f, 0x9b, 0xac, 0xf6, 0xd5, 0xb1, 0x7c, 0x19, 0x93, 0xe3, - 0xdb, 0x8a, 0xff, 0xcd, 0xa8, 0xf8, 0x9f, 0xa7, 0x60, 0x39, 0x34, 0x0e, 0x85, 0x57, 0xd2, 0xaa, - 0x3f, 0x98, 0xb4, 0xea, 0xea, 0xa4, 0x55, 0xe5, 0x87, 0xdf, 0x9a, 0xf6, 0x1b, 0x65, 0xda, 0x4d, - 0xdd, 0x94, 0xcb, 0xb0, 0x53, 0xad, 0x50, 0x15, 0xf2, 0x81, 0xd5, 0xc5, 0x5e, 0x41, 0x56, 0x9d, - 0x02, 0x0f, 0x61, 0xf3, 0x6d, 0xb8, 0x1e, 0x7d, 0x71, 0x5a, 0x0f, 0xbf, 0xa9, 0x43, 0x96, 0xd2, - 0x84, 0xae, 0x53, 0xd3, 0xe2, 0xfa, 0xb4, 0x2e, 0xfb, 0x3f, 0x45, 0x69, 0xbe, 0x11, 0x4f, 0x3e, - 0x6a, 0x33, 0x2c, 0x29, 0x46, 0xac, 0xa4, 0x30, 0x48, 0x07, 0x38, 0x7b, 0xcd, 0x92, 0x30, 0xb4, - 0x36, 0x07, 0xb1, 0x2c, 0x93, 0xf0, 0x2d, 0xea, 0xa4, 0xa4, 0xb8, 0x61, 0x27, 0x25, 0x41, 0x4c, - 0x61, 0x34, 0x66, 0xea, 0xf1, 0x84, 0x80, 0x28, 0xb1, 0xa5, 0xa7, 0x24, 0xb6, 0x4c, 0x94, 0xd8, - 0x5e, 0x85, 0xa7, 0x27, 0x4e, 0x54, 0xb7, 0xc7, 0xb4, 0xad, 0x91, 0x4a, 0x65, 0x11, 0xc2, 0xbc, - 0x0d, 0x79, 0xfd, 0x09, 0x5d, 0x65, 0x14, 0xa6, 0x56, 0x5a, 0x4f, 0x9f, 0x9a, 0xcc, 0x03, 0xb8, - 0x31, 0x76, 0x5c, 0x4c, 0xdd, 0x1b, 0xe3, 0x07, 0x16, 0xeb, 0x8b, 0x51, 0x63, 0xa4, 0x76, 0xe2, - 0x32, 0x6c, 0x43, 0x86, 0x4a, 0x1a, 0xbb, 0x0b, 0xb9, 0x16, 0xf5, 0x06, 0xfa, 0xbb, 0x28, 0x56, - 0xe5, 0x6b, 0xc0, 0xc5, 0xad, 0x1a, 0x17, 0xbe, 0x3b, 0xf4, 0xda, 0x82, 0x6a, 0x04, 0xd7, 0xf4, - 0xe6, 0x11, 0x94, 0x4e, 0x86, 0x7e, 0xd4, 0x32, 0xbf, 0x09, 0x73, 0xd4, 0xb4, 0xf8, 0xdb, 0xa3, - 0xa6, 0x9a, 0xcd, 0x53, 0xeb, 0xf3, 0x31, 0x07, 0x44, 0xea, 0x5d, 0xa4, 0xe0, 0xc2, 0xf2, 0x5d, - 0x87, 0x27, 0xc9, 0xcd, 0xdf, 0x19, 0x50, 0x46, 0x12, 0x2a, 0x59, 0xda, 0x7a, 0x2f, 0x87, 0x7d, - 0x38, 0x5a, 0xbb, 0xb4, 0xfd, 0x14, 0xce, 0xd1, 0x7f, 0xff, 0x62, 0x75, 0xee, 0xc4, 0x13, 0x56, - 0xaf, 0xe7, 0xb6, 0x25, 0xb5, 0x6e, 0xc0, 0xbf, 0x0b, 0x29, 0xbb, 0x23, 0x1b, 0x9b, 0x4b, 0x69, - 0x91, 0x82, 0xdd, 0x01, 0x90, 0x39, 0x67, 0xc7, 0x0a, 0xac, 0x4a, 0xfa, 0x2a, 0xfa, 0x18, 0xa1, - 0x79, 0x28, 0x45, 0x94, 0x9a, 0x50, 0x22, 0xfe, 0x0f, 0x2a, 0xbc, 0x09, 0xa0, 0xde, 0x1a, 0xb0, - 0x4a, 0x2f, 0x25, 0x66, 0x8e, 0x92, 0xbe, 0x94, 0xf9, 0x26, 0x14, 0x0e, 0x6c, 0xe7, 0xbc, 0xd1, - 0xb3, 0xdb, 0x38, 0x12, 0x65, 0x7a, 0xb6, 0x73, 0xae, 0xcf, 0x5a, 0x9e, 0x3c, 0x0b, 0xcf, 0xa8, - 0xe1, 0x07, 0x5c, 0x52, 0x9a, 0x3f, 0x33, 0x80, 0x21, 0x52, 0x0f, 0x1f, 0x51, 0x5d, 0x97, 0xee, - 0x6f, 0xc4, 0xdd, 0xbf, 0x02, 0xb9, 0xae, 0xe7, 0x0e, 0x07, 0xdb, 0x3a, 0x2c, 0x34, 0x88, 0xf4, - 0x3d, 0x7a, 0x6a, 0x90, 0xdd, 0x9b, 0x04, 0xfe, 0xe3, 0x70, 0xf9, 0xa5, 0x01, 0x37, 0x62, 0x42, - 0x34, 0x86, 0xfd, 0xbe, 0xe5, 0x8d, 0xfe, 0x3f, 0xb2, 0xfc, 0xc1, 0x80, 0x6b, 0x09, 0x85, 0x44, - 0x71, 0x2b, 0xfc, 0xc0, 0xee, 0x63, 0x4e, 0x24, 0x49, 0xf2, 0x3c, 0x42, 0x24, 0x9b, 0x78, 0xd9, - 0xf7, 0xc5, 0x9a, 0xf8, 0xe7, 0x61, 0x9e, 0xdc, 0xb9, 0x11, 0x92, 0x48, 0xd1, 0xc6, 0xb0, 0xac, - 0x16, 0x8d, 0x88, 0x69, 0xb2, 0xe0, 0xf5, 0x44, 0x0b, 0x3f, 0x31, 0x20, 0x7e, 0x1f, 0x4a, 0xdc, - 0xfa, 0xc9, 0x5b, 0xb6, 0x1f, 0xb8, 0x5d, 0xcf, 0xea, 0xa3, 0x93, 0xb4, 0x86, 0xed, 0x73, 0x21, - 0xe7, 0x88, 0x34, 0x57, 0x10, 0xde, 0xbd, 0x1d, 0x93, 0x4c, 0x02, 0xe6, 0xdb, 0x90, 0xd7, 0x4d, - 0xf0, 0x94, 0xb9, 0xe6, 0xa5, 0xe4, 0x5c, 0xb3, 0x94, 0x9c, 0xa5, 0xde, 0x3d, 0xc0, 0xe1, 0xc5, - 0x6e, 0xeb, 0x0c, 0xf4, 0x1b, 0x03, 0x8a, 0x31, 0x11, 0xd9, 0x36, 0x2c, 0xf6, 0xac, 0x40, 0x38, - 0xed, 0xd1, 0xd9, 0x03, 0x2d, 0x9e, 0xf2, 0xca, 0x68, 0x42, 0x8a, 0xcb, 0xce, 0xcb, 0x8a, 0x3e, - 0xba, 0xcd, 0xf7, 0x20, 0xeb, 0x0b, 0xcf, 0x56, 0xe1, 0x1d, 0xcf, 0x5a, 0x61, 0xef, 0xae, 0x08, - 0xf0, 0xe2, 0x32, 0x5f, 0x28, 0xc5, 0x2a, 0xc8, 0xfc, 0x6b, 0xd2, 0xbb, 0x95, 0x63, 0x4d, 0x8e, - 0x5c, 0x8f, 0xb0, 0xd6, 0xec, 0x54, 0x6b, 0x45, 0xf2, 0xa5, 0x1e, 0x25, 0x5f, 0x19, 0x52, 0x83, - 0xbb, 0x77, 0xd5, 0xc0, 0x82, 0x4b, 0x89, 0xb9, 0x43, 0x8e, 0x47, 0x98, 0x3b, 0x12, 0xb3, 0xa9, - 0xba, 0x74, 0x5c, 0x12, 0xe6, 0xce, 0xa6, 0x6a, 0xc7, 0x71, 0x69, 0xbe, 0x0f, 0xd5, 0x69, 0x71, - 0xa2, 0x5c, 0xf4, 0x2e, 0x14, 0x7c, 0x42, 0xd9, 0x62, 0x32, 0x05, 0x4c, 0xf9, 0x2e, 0xa2, 0x36, - 0x7f, 0x6b, 0xc0, 0x5c, 0xc2, 0xb0, 0x89, 0xea, 0x93, 0x51, 0xd5, 0xa7, 0x04, 0x86, 0x43, 0xca, - 0x48, 0x71, 0xc3, 0x41, 0xe8, 0x3e, 0xe9, 0xdb, 0xe0, 0xc6, 0x7d, 0x84, 0xe4, 0xa0, 0x52, 0xe0, - 0x86, 0x8f, 0x50, 0x8b, 0x2e, 0x97, 0xe7, 0x46, 0x0b, 0xa1, 0x8e, 0xba, 0x98, 0xd1, 0xa1, 0x09, - 0x31, 0xb0, 0x82, 0xa1, 0xec, 0x8f, 0x32, 0x5c, 0x41, 0x78, 0xe2, 0xb9, 0xed, 0x74, 0xa8, 0x23, - 0xca, 0x70, 0x5a, 0x9b, 0x42, 0x3e, 0x37, 0x2a, 0xc1, 0x31, 0xcd, 0x62, 0xbb, 0xe3, 0x09, 0x7f, - 0xd8, 0x0b, 0x9a, 0x51, 0x71, 0x8c, 0x61, 0xb0, 0xbd, 0x90, 0x90, 0x72, 0x9b, 0xea, 0xd4, 0x18, - 0x22, 0x0a, 0xae, 0x28, 0x31, 0x0b, 0x2e, 0x4e, 0xec, 0xa2, 0x9b, 0xf4, 0xac, 0x96, 0xe8, 0xc5, - 0xfa, 0x83, 0x08, 0x81, 0x72, 0x10, 0x70, 0x1a, 0xab, 0xc7, 0x31, 0x0c, 0xdb, 0x80, 0xd9, 0x40, - 0xbb, 0xc6, 0xea, 0xe5, 0x32, 0x9c, 0xb8, 0xb6, 0x13, 0xf0, 0xd9, 0xc0, 0xc7, 0x18, 0x5a, 0x9a, - 0xbe, 0x4d, 0xc6, 0xb0, 0x95, 0x10, 0x73, 0x9c, 0xd6, 0xe8, 0x1d, 0x17, 0x56, 0x8f, 0x0e, 0x36, - 0x38, 0x2e, 0x71, 0xe6, 0x13, 0x0f, 0x45, 0x7f, 0xd0, 0xb3, 0xbc, 0xa6, 0x7a, 0x1f, 0x4a, 0xd1, - 0x4b, 0xfc, 0x38, 0x9a, 0xbd, 0x00, 0x65, 0x8d, 0xd2, 0xef, 0xc5, 0xca, 0x39, 0x27, 0xf0, 0xe6, - 0x9f, 0x0c, 0x58, 0xa4, 0xb7, 0x5f, 0x6e, 0x39, 0x5d, 0x71, 0x75, 0x52, 0x0e, 0x93, 0xac, 0x4a, - 0x34, 0x89, 0x24, 0x2b, 0x43, 0x93, 0xde, 0x96, 0xb1, 0x8d, 0x0d, 0xc4, 0x40, 0x9d, 0x49, 0x6b, - 0x4c, 0xe8, 0xfe, 0x03, 0xcb, 0xeb, 0xec, 0xef, 0xa8, 0x74, 0xac, 0x41, 0xd4, 0x34, 0x2d, 0x65, - 0x30, 0xca, 0xce, 0x3b, 0x86, 0x49, 0xfe, 0x46, 0x90, 0x1b, 0xff, 0x8d, 0xc0, 0x07, 0x16, 0x17, - 0x5f, 0xc5, 0xca, 0x8b, 0x61, 0xf0, 0xca, 0x40, 0xb9, 0x16, 0xe5, 0x37, 0xbb, 0x2f, 0x1a, 0xb4, - 0x15, 0x86, 0xef, 0xe3, 0x3f, 0xdd, 0x6d, 0x41, 0xb6, 0x61, 0xe1, 0xb4, 0xce, 0xbe, 0x03, 0x25, - 0x34, 0x97, 0x1f, 0x58, 0xfd, 0xc1, 0x59, 0xdf, 0x57, 0xe1, 0x53, 0x0c, 0x71, 0xf2, 0x9d, 0x5e, - 0xa6, 0x5a, 0x83, 0x6c, 0xa9, 0x52, 0xea, 0xc7, 0x06, 0x40, 0x24, 0x0b, 0xbb, 0x0b, 0x59, 0x72, - 0xae, 0xc9, 0xc8, 0x9e, 0x7c, 0xd3, 0x50, 0xbf, 0x28, 0xa8, 0x0f, 0xd8, 0x06, 0xe4, 0x7c, 0x12, - 0x46, 0x67, 0xd2, 0x85, 0x48, 0x7c, 0xc2, 0x2b, 0x7a, 0x4d, 0xc5, 0x56, 0xa1, 0x38, 0xf0, 0xdc, - 0xfe, 0x99, 0x3a, 0x50, 0x3e, 0x0d, 0x02, 0xa2, 0x0e, 0x08, 0xf3, 0xc2, 0x87, 0xb0, 0x30, 0xd6, - 0xb0, 0xb1, 0x12, 0xe4, 0x8f, 0x8e, 0xcf, 0x76, 0x39, 0x3f, 0xe6, 0xe5, 0x19, 0x76, 0x0d, 0x16, - 0x0e, 0xb7, 0x3e, 0x38, 0x3b, 0xd8, 0x3f, 0xdd, 0x3d, 0x6b, 0xf2, 0xad, 0x7b, 0xbb, 0x8d, 0xb2, - 0x81, 0x48, 0x5a, 0x9f, 0x35, 0x8f, 0x8f, 0xcf, 0x0e, 0xb6, 0xf8, 0xde, 0x6e, 0x79, 0x96, 0x2d, - 0xc2, 0xdc, 0x7b, 0x47, 0xef, 0x1c, 0x1d, 0xbf, 0x7f, 0xa4, 0x3e, 0x4e, 0xd5, 0x7f, 0x65, 0x40, - 0x16, 0xd9, 0x0b, 0x8f, 0xfd, 0x10, 0x0a, 0x61, 0xdb, 0xc7, 0x6e, 0x24, 0xba, 0xc5, 0x78, 0x2b, - 0x58, 0x7d, 0x2a, 0xb1, 0xa5, 0xad, 0x6c, 0xce, 0xb0, 0x2d, 0x28, 0x86, 0xc4, 0xa7, 0xf5, 0xff, - 0x86, 0x45, 0xfd, 0x9f, 0x06, 0x94, 0x95, 0x81, 0xf7, 0x84, 0x23, 0x3c, 0x2b, 0x70, 0x43, 0xc1, - 0xa8, 0x67, 0x1b, 0xe3, 0x1a, 0x6f, 0x00, 0x2f, 0x17, 0x6c, 0x1f, 0x60, 0x4f, 0x04, 0xba, 0x5e, - 0x2e, 0x4f, 0x4f, 0x10, 0x92, 0xc7, 0x33, 0x97, 0x64, 0x0f, 0xcd, 0x6a, 0x0f, 0x20, 0xf2, 0x70, - 0x16, 0xe5, 0xbb, 0x89, 0xa8, 0xad, 0x2e, 0x4f, 0xdd, 0x0b, 0x6f, 0xfa, 0xfb, 0x34, 0xe4, 0x70, - 0xc3, 0x16, 0x1e, 0x7b, 0x0b, 0xe6, 0x7e, 0x64, 0x3b, 0x9d, 0xf0, 0xe7, 0x2e, 0x36, 0xe5, 0xf7, - 0x31, 0xcd, 0xb6, 0x3a, 0x6d, 0x2b, 0x66, 0x82, 0x92, 0x7e, 0x40, 0x6f, 0x0b, 0x27, 0x60, 0x97, - 0xfc, 0x6a, 0x53, 0x7d, 0x7a, 0x02, 0x1f, 0xb2, 0xd8, 0x85, 0x62, 0xec, 0x17, 0xa1, 0xb8, 0xb6, - 0x26, 0x7e, 0x27, 0xba, 0x8a, 0xcd, 0x1e, 0x40, 0x34, 0x45, 0xb2, 0x2b, 0xde, 0x93, 0xaa, 0xcb, - 0x53, 0xf7, 0x42, 0x46, 0xef, 0xe8, 0x2b, 0xc9, 0x71, 0xf4, 0x4a, 0x56, 0xcf, 0x4e, 0x1d, 0x6f, - 0x63, 0xcc, 0x4e, 0x61, 0x61, 0x6c, 0x7a, 0x63, 0x8f, 0x7a, 0x14, 0xa9, 0xae, 0x5d, 0x4e, 0x10, - 0xf2, 0xfd, 0x71, 0x6c, 0x66, 0xd6, 0x53, 0xe1, 0xa3, 0x39, 0x9b, 0x97, 0x11, 0xc4, 0x65, 0xae, - 0xff, 0x2b, 0x05, 0xe5, 0x46, 0xe0, 0x09, 0xab, 0x6f, 0x3b, 0x5d, 0xed, 0x32, 0x6f, 0x40, 0x56, - 0x3d, 0xa5, 0x3c, 0xae, 0x89, 0x37, 0x0d, 0x8c, 0x87, 0x27, 0x62, 0x9b, 0x4d, 0x83, 0x1d, 0x3e, - 0x41, 0xeb, 0x6c, 0x1a, 0xec, 0x83, 0xaf, 0xc7, 0x3e, 0x9b, 0x06, 0xfb, 0xf0, 0xeb, 0xb3, 0xd0, - 0xa6, 0xc1, 0x4e, 0x60, 0x51, 0xe5, 0x8a, 0x27, 0x92, 0x1d, 0x36, 0x8d, 0xfa, 0x1f, 0x0d, 0xc8, - 0xe9, 0x8c, 0x75, 0x36, 0xb5, 0xb3, 0x36, 0xaf, 0xea, 0x37, 0xd5, 0x31, 0xcf, 0x5d, 0x49, 0xf3, - 0xc4, 0xb3, 0xda, 0x76, 0xe5, 0x93, 0x2f, 0x57, 0x8c, 0xcf, 0xbe, 0x5c, 0x31, 0xfe, 0xf1, 0xe5, - 0x8a, 0xf1, 0xeb, 0xaf, 0x56, 0x66, 0x3e, 0xfb, 0x6a, 0x65, 0xe6, 0xf3, 0xaf, 0x56, 0x66, 0x5a, - 0x59, 0xfa, 0x77, 0x86, 0x57, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x39, 0xe8, 0x42, 0x13, 0x4f, - 0x21, 0x00, 0x00, + // 2646 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcd, 0x6e, 0x23, 0xc7, + 0xf1, 0xd7, 0x88, 0x1f, 0x22, 0x8b, 0x94, 0x44, 0xf5, 0xae, 0x65, 0x2e, 0xd7, 0xd6, 0xea, 0x3f, + 0x5e, 0xf8, 0xaf, 0xf8, 0x43, 0xd2, 0xd2, 0xbb, 0xb0, 0xd7, 0x4e, 0x1c, 0xac, 0x56, 0xca, 0x5a, + 0xb6, 0xbe, 0xdc, 0x94, 0x65, 0x23, 0x30, 0x20, 0x8c, 0xc8, 0x5e, 0xee, 0x40, 0xe4, 0x0c, 0x3d, + 0x33, 0x54, 0x96, 0x39, 0x06, 0x48, 0x80, 0x00, 0x39, 0xe4, 0x90, 0x1c, 0x7c, 0xcc, 0x29, 0xc8, + 0x39, 0x8f, 0x10, 0x20, 0x30, 0x10, 0xc4, 0x30, 0x90, 0x8b, 0x91, 0x83, 0x11, 0xd8, 0x87, 0x3c, + 0x40, 0x5e, 0x20, 0xa8, 0xea, 0xee, 0x99, 0x1e, 0x72, 0xa4, 0xf5, 0x26, 0x6b, 0xc4, 0x07, 0x9f, + 0xd4, 0x55, 0x5d, 0x5d, 0x5d, 0xdd, 0x55, 0xfd, 0xab, 0xaa, 0xa1, 0xe0, 0xe9, 0xc1, 0x69, 0x77, + 0x2d, 0x12, 0xfd, 0x81, 0x3f, 0x38, 0x91, 0x7f, 0x57, 0x07, 0x81, 0x1f, 0xf9, 0x6c, 0x46, 0x31, + 0x1b, 0x8b, 0x6d, 0xbf, 0xdf, 0xf7, 0xbd, 0xb5, 0xb3, 0x1b, 0x6b, 0x72, 0x24, 0x05, 0x1a, 0x2f, + 0x77, 0xdd, 0xe8, 0xc1, 0xf0, 0x64, 0xb5, 0xed, 0xf7, 0xd7, 0xba, 0x7e, 0xd7, 0x5f, 0x23, 0xf6, + 0xc9, 0xf0, 0x3e, 0x51, 0x44, 0xd0, 0x48, 0x89, 0x5f, 0x8e, 0x02, 0xa7, 0x2d, 0x50, 0x0b, 0x0d, + 0x24, 0xd7, 0xfe, 0x85, 0x05, 0xb5, 0x43, 0xa4, 0x37, 0x46, 0xdb, 0x9b, 0x5c, 0x7c, 0x34, 0x14, + 0x61, 0xc4, 0xea, 0x30, 0x43, 0x32, 0xdb, 0x9b, 0x75, 0x6b, 0xd9, 0x5a, 0xa9, 0x72, 0x4d, 0xb2, + 0x25, 0x80, 0x93, 0x9e, 0xdf, 0x3e, 0x6d, 0x45, 0x4e, 0x10, 0xd5, 0xa7, 0x97, 0xad, 0x95, 0x32, + 0x37, 0x38, 0xac, 0x01, 0x25, 0xa2, 0xb6, 0xbc, 0x4e, 0x3d, 0x47, 0xb3, 0x31, 0xcd, 0x9e, 0x81, + 0xf2, 0x47, 0x43, 0x11, 0x8c, 0x76, 0xfd, 0x8e, 0xa8, 0x17, 0x68, 0x32, 0x61, 0xd8, 0x1e, 0x2c, + 0x18, 0x76, 0x84, 0x03, 0xdf, 0x0b, 0x05, 0xbb, 0x0e, 0x05, 0xda, 0x99, 0xcc, 0xa8, 0x34, 0xe7, + 0x56, 0xd5, 0x9d, 0xac, 0x92, 0x28, 0x97, 0x93, 0xec, 0x15, 0x98, 0xe9, 0x8b, 0x28, 0x70, 0xdb, + 0x21, 0x59, 0x54, 0x69, 0x5e, 0x49, 0xcb, 0xa1, 0xca, 0x5d, 0x29, 0xc0, 0xb5, 0xa4, 0xcd, 0x8c, + 0x73, 0xab, 0x49, 0xfb, 0xd3, 0x69, 0x98, 0x6d, 0x09, 0x27, 0x68, 0x3f, 0xd0, 0x37, 0xf1, 0x3a, + 0xe4, 0x0f, 0x9d, 0x6e, 0x58, 0xb7, 0x96, 0x73, 0x2b, 0x95, 0xe6, 0x72, 0xac, 0x37, 0x25, 0xb5, + 0x8a, 0x22, 0x5b, 0x5e, 0x14, 0x8c, 0x36, 0xf2, 0x9f, 0x7c, 0x71, 0x6d, 0x8a, 0xd3, 0x1a, 0x76, + 0x1d, 0x66, 0x77, 0x5d, 0x6f, 0x73, 0x18, 0x38, 0x91, 0xeb, 0x7b, 0xbb, 0xd2, 0xb8, 0x59, 0x9e, + 0x66, 0x92, 0x94, 0xf3, 0xd0, 0x90, 0xca, 0x29, 0x29, 0x93, 0xc9, 0x2e, 0x43, 0x61, 0xc7, 0xed, + 0xbb, 0x51, 0x3d, 0x4f, 0xb3, 0x92, 0x40, 0x6e, 0x48, 0x8e, 0x28, 0x48, 0x2e, 0x11, 0xac, 0x06, + 0x39, 0xe1, 0x75, 0xea, 0x45, 0xe2, 0xe1, 0x10, 0xe5, 0xde, 0xc5, 0x8b, 0xae, 0x97, 0xe8, 0xd6, + 0x25, 0xc1, 0x56, 0x60, 0xbe, 0x35, 0x70, 0xbc, 0xf0, 0x40, 0x04, 0xf8, 0xb7, 0x25, 0xa2, 0x7a, + 0x99, 0xd6, 0x8c, 0xb3, 0x1b, 0xaf, 0x42, 0x39, 0x3e, 0x22, 0xaa, 0x3f, 0x15, 0x23, 0xf2, 0x48, + 0x99, 0xe3, 0x10, 0xd5, 0x9f, 0x39, 0xbd, 0xa1, 0x50, 0xf1, 0x20, 0x89, 0xd7, 0xa7, 0x5f, 0xb3, + 0xec, 0x3f, 0xe7, 0x80, 0xc9, 0xab, 0xda, 0xc0, 0x28, 0xd0, 0xb7, 0x7a, 0x13, 0xca, 0xa1, 0xbe, + 0x40, 0xe5, 0xda, 0xc5, 0xec, 0xab, 0xe5, 0x89, 0x20, 0x46, 0x25, 0xc5, 0xd2, 0xf6, 0xa6, 0xda, + 0x48, 0x93, 0x18, 0x59, 0x74, 0xf4, 0x03, 0xa7, 0x2b, 0xd4, 0xfd, 0x25, 0x0c, 0xbc, 0xe1, 0x81, + 0xd3, 0x15, 0xe1, 0xa1, 0x2f, 0x55, 0xab, 0x3b, 0x4c, 0x33, 0x31, 0x72, 0x85, 0xd7, 0xf6, 0x3b, + 0xae, 0xd7, 0x55, 0xc1, 0x19, 0xd3, 0xa8, 0xc1, 0xf5, 0x3a, 0xe2, 0x21, 0xaa, 0x6b, 0xb9, 0x3f, + 0x15, 0xea, 0x6e, 0xd3, 0x4c, 0x66, 0x43, 0x35, 0xf2, 0x23, 0xa7, 0xc7, 0x45, 0xdb, 0x0f, 0x3a, + 0x61, 0x7d, 0x86, 0x84, 0x52, 0x3c, 0x94, 0xe9, 0x38, 0x91, 0xb3, 0xa5, 0x77, 0x92, 0x0e, 0x49, + 0xf1, 0xf0, 0x9c, 0x67, 0x22, 0x08, 0x5d, 0xdf, 0x23, 0x7f, 0x94, 0xb9, 0x26, 0x19, 0x83, 0x7c, + 0x88, 0xdb, 0xc3, 0xb2, 0xb5, 0x92, 0xe7, 0x34, 0xc6, 0x17, 0x79, 0xdf, 0xf7, 0x23, 0x11, 0x90, + 0x61, 0x15, 0xda, 0xd3, 0xe0, 0xb0, 0x4d, 0xa8, 0x75, 0x44, 0xc7, 0x6d, 0x3b, 0x91, 0xe8, 0xdc, + 0xf5, 0x7b, 0xc3, 0xbe, 0x17, 0xd6, 0xab, 0x14, 0xcd, 0xf5, 0xf8, 0xca, 0x37, 0xd3, 0x02, 0x7c, + 0x62, 0x85, 0xfd, 0x27, 0x0b, 0xe6, 0xc7, 0xa4, 0xd8, 0x4d, 0x28, 0x84, 0x6d, 0x7f, 0x20, 0x6f, + 0x7c, 0xae, 0xb9, 0x74, 0x9e, 0xba, 0xd5, 0x16, 0x4a, 0x71, 0x29, 0x8c, 0x67, 0xf0, 0x9c, 0xbe, + 0x8e, 0x15, 0x1a, 0xb3, 0x1b, 0x90, 0x8f, 0x46, 0x03, 0xf9, 0xca, 0xe7, 0x9a, 0xcf, 0x9e, 0xab, + 0xe8, 0x70, 0x34, 0x10, 0x9c, 0x44, 0xed, 0x6b, 0x50, 0x20, 0xb5, 0xac, 0x04, 0xf9, 0xd6, 0xc1, + 0x9d, 0xbd, 0xda, 0x14, 0xab, 0x42, 0x89, 0x6f, 0xb5, 0xf6, 0xdf, 0xe3, 0x77, 0xb7, 0x6a, 0x96, + 0xcd, 0x20, 0x8f, 0xe2, 0x0c, 0xa0, 0xd8, 0x3a, 0xe4, 0xdb, 0x7b, 0xf7, 0x6a, 0x53, 0xf6, 0x43, + 0x98, 0xd3, 0xd1, 0xa5, 0x00, 0xe6, 0x26, 0x14, 0x09, 0x43, 0xf4, 0x0b, 0x7f, 0x26, 0x8d, 0x1c, + 0x52, 0x7a, 0x57, 0x44, 0x0e, 0x7a, 0x88, 0x2b, 0x59, 0xb6, 0x3e, 0x0e, 0x38, 0xe3, 0xd1, 0x3b, + 0x81, 0x36, 0x7f, 0xcb, 0xc1, 0xa5, 0x0c, 0x8d, 0xe3, 0x48, 0x5b, 0x4e, 0x90, 0x76, 0x05, 0xe6, + 0x03, 0xdf, 0x8f, 0x5a, 0x22, 0x38, 0x73, 0xdb, 0x62, 0x2f, 0xb9, 0xb2, 0x71, 0x36, 0x46, 0x27, + 0xb2, 0x48, 0x3d, 0xc9, 0x49, 0xe0, 0x4d, 0x33, 0xd9, 0x4b, 0xb0, 0x40, 0x4f, 0xe2, 0xd0, 0xed, + 0x8b, 0xf7, 0x3c, 0xf7, 0xe1, 0x9e, 0xe3, 0xf9, 0xf4, 0x12, 0xf2, 0x7c, 0x72, 0x02, 0xa3, 0xaa, + 0x93, 0x40, 0x92, 0x84, 0x17, 0x83, 0xc3, 0x5e, 0x80, 0x99, 0x50, 0x61, 0x46, 0x91, 0x6e, 0xa0, + 0x96, 0xdc, 0x80, 0xe4, 0x73, 0x2d, 0xc0, 0x5e, 0x82, 0x92, 0x1a, 0xe2, 0x9b, 0xc8, 0x65, 0x0a, + 0xc7, 0x12, 0x8c, 0x43, 0x35, 0x94, 0x87, 0x6b, 0x45, 0x4e, 0x14, 0xd6, 0x4b, 0xb4, 0x62, 0xf5, + 0x22, 0xbf, 0xac, 0xb6, 0x8c, 0x05, 0x04, 0x52, 0x3c, 0xa5, 0xa3, 0x71, 0x04, 0x0b, 0x13, 0x22, + 0x19, 0x38, 0xf6, 0xa2, 0x89, 0x63, 0x95, 0xe6, 0x53, 0x86, 0x53, 0x93, 0xc5, 0x26, 0xbc, 0xed, + 0x40, 0xd5, 0x9c, 0x22, 0x1c, 0x1a, 0x38, 0xde, 0x5d, 0x7f, 0xe8, 0x45, 0xa4, 0x18, 0x71, 0x48, + 0x33, 0xf0, 0x4e, 0x45, 0x10, 0xf8, 0x81, 0x9c, 0x96, 0xc9, 0xc0, 0xe0, 0xd8, 0x3f, 0xb7, 0x60, + 0x46, 0xdd, 0x07, 0x7b, 0x0e, 0x0a, 0xb8, 0x50, 0x87, 0xe5, 0x6c, 0xea, 0xc2, 0xb8, 0x9c, 0xc3, + 0xe0, 0xe9, 0x3b, 0x51, 0xfb, 0x81, 0xe8, 0x28, 0x6d, 0x9a, 0x64, 0x6f, 0x00, 0x38, 0x51, 0x14, + 0xb8, 0x27, 0xc3, 0x48, 0x60, 0x46, 0x41, 0x1d, 0x57, 0x63, 0x1d, 0xaa, 0x8a, 0x38, 0xbb, 0xb1, + 0xfa, 0x8e, 0x18, 0x1d, 0xe1, 0x69, 0xb8, 0x21, 0x8e, 0x6f, 0x3d, 0x8f, 0xdb, 0xb0, 0x45, 0x28, + 0xe2, 0x46, 0x71, 0x6c, 0x2a, 0x2a, 0xf3, 0x09, 0x67, 0x86, 0x57, 0xee, 0xbc, 0xf0, 0xba, 0x0e, + 0xb3, 0x3a, 0x98, 0x90, 0x0e, 0x55, 0x20, 0xa6, 0x99, 0x63, 0xa7, 0x28, 0x3c, 0xde, 0x29, 0x3e, + 0x8e, 0x73, 0xb9, 0x7a, 0x8c, 0xf8, 0xa2, 0x5c, 0x2f, 0x1c, 0x88, 0x76, 0x24, 0x3a, 0x87, 0xfa, + 0xd1, 0x53, 0xbe, 0x1b, 0x63, 0xb3, 0xe7, 0x61, 0x2e, 0x66, 0x6d, 0x8c, 0x70, 0xf3, 0x69, 0xb2, + 0x6f, 0x8c, 0xcb, 0x96, 0xa1, 0x42, 0xe8, 0x4e, 0xc9, 0x4d, 0x67, 0x6e, 0x93, 0x85, 0x07, 0x6d, + 0xfb, 0xfd, 0x41, 0x4f, 0x44, 0xa2, 0xf3, 0xb6, 0x7f, 0x12, 0xea, 0xdc, 0x93, 0x62, 0x62, 0xdc, + 0xd0, 0x22, 0x92, 0x90, 0x8f, 0x2d, 0x61, 0xa0, 0xdd, 0x89, 0x4a, 0x69, 0x4e, 0x91, 0xcc, 0x19, + 0x67, 0xa7, 0xec, 0xa6, 0x1c, 0x4e, 0x39, 0xc8, 0xb4, 0x9b, 0xb8, 0xf6, 0xbb, 0xf8, 0x1e, 0xf0, + 0x6a, 0x30, 0xab, 0xeb, 0xa4, 0x7c, 0x59, 0xc3, 0xb9, 0x74, 0xb6, 0x82, 0xeb, 0xb8, 0xc4, 0xc8, + 0x65, 0x94, 0x18, 0xf9, 0xb8, 0xc4, 0xb0, 0x3f, 0xcd, 0xc1, 0x62, 0xa2, 0x33, 0x95, 0xed, 0x5f, + 0x9b, 0xcc, 0xf6, 0x8d, 0x31, 0xbc, 0x34, 0xec, 0xf8, 0x2e, 0xe3, 0x7f, 0x3b, 0x32, 0xfe, 0xe7, + 0x39, 0xb8, 0x1a, 0x3b, 0x87, 0x9e, 0x57, 0xda, 0xab, 0x3f, 0x98, 0xf4, 0xea, 0xb5, 0x49, 0xaf, + 0xca, 0x85, 0xdf, 0xb9, 0xf6, 0x5b, 0xe5, 0xda, 0x75, 0x5d, 0x94, 0xcb, 0x67, 0xa7, 0x4a, 0xa1, + 0x06, 0x94, 0x22, 0xa7, 0x8b, 0xb5, 0x82, 0xcc, 0x3a, 0x65, 0x1e, 0xd3, 0xf6, 0xdb, 0x70, 0x39, + 0x59, 0x71, 0xd4, 0x8c, 0xd7, 0x34, 0xa1, 0x48, 0x30, 0xa1, 0xf3, 0x54, 0xd6, 0xbb, 0x3e, 0x6a, + 0xca, 0xfa, 0x4f, 0x49, 0xda, 0x6f, 0x98, 0xe0, 0xa3, 0x26, 0xe3, 0x94, 0x62, 0x19, 0x29, 0x85, + 0x41, 0x3e, 0xc2, 0xde, 0x6b, 0x9a, 0x8c, 0xa1, 0xb1, 0x3d, 0x30, 0x50, 0x26, 0x15, 0x5b, 0x54, + 0x49, 0x49, 0x73, 0xe3, 0x4a, 0x4a, 0x92, 0x08, 0x61, 0xd4, 0x66, 0xea, 0xf6, 0x84, 0x88, 0x04, + 0xd8, 0xf2, 0x19, 0xc0, 0x56, 0x48, 0x80, 0xed, 0x55, 0x78, 0x7a, 0x62, 0x47, 0x75, 0x7a, 0x84, + 0x6d, 0xcd, 0x54, 0x57, 0x96, 0x30, 0xec, 0x9b, 0x50, 0xd2, 0x4b, 0xe8, 0x28, 0xa3, 0x18, 0x5a, + 0x69, 0x9c, 0xdd, 0x35, 0xd9, 0x3b, 0x70, 0x65, 0x6c, 0x3b, 0xe3, 0xba, 0xd7, 0xc6, 0x37, 0xac, + 0x34, 0x17, 0x92, 0xc2, 0x48, 0xcd, 0x98, 0x36, 0x6c, 0x40, 0x81, 0x52, 0x1a, 0xbb, 0x0d, 0x33, + 0x27, 0x54, 0x1b, 0xe8, 0x75, 0xc9, 0x5b, 0x95, 0x5f, 0x03, 0xce, 0x6e, 0xac, 0x72, 0x11, 0xfa, + 0xc3, 0xa0, 0x2d, 0x28, 0x47, 0x70, 0x2d, 0x6f, 0xef, 0x41, 0xf5, 0x60, 0x18, 0x26, 0x25, 0xf3, + 0x9b, 0x30, 0x4b, 0x45, 0x4b, 0xb8, 0x31, 0x3a, 0x54, 0xbd, 0x79, 0x6e, 0x65, 0xce, 0x08, 0x40, + 0x94, 0xde, 0x42, 0x09, 0x2e, 0x9c, 0xd0, 0xf7, 0x78, 0x5a, 0xdc, 0xfe, 0x9d, 0x05, 0x35, 0x14, + 0xa1, 0x94, 0xa5, 0xbd, 0xf7, 0x72, 0x5c, 0x87, 0xa3, 0xb7, 0xab, 0x1b, 0x4f, 0x61, 0x1f, 0xfd, + 0xf7, 0x2f, 0xae, 0xcd, 0x1e, 0x04, 0xc2, 0xe9, 0xf5, 0xfc, 0xb6, 0x94, 0xd6, 0x05, 0xf8, 0xff, + 0x43, 0xce, 0xed, 0xc8, 0xc2, 0xe6, 0x5c, 0x59, 0x94, 0x60, 0xb7, 0x00, 0x24, 0xe6, 0x6c, 0x3a, + 0x91, 0x53, 0xcf, 0x5f, 0x24, 0x6f, 0x08, 0xda, 0xbb, 0xd2, 0x44, 0x79, 0x13, 0xca, 0xc4, 0xff, + 0xe2, 0x0a, 0xaf, 0x03, 0xa8, 0x6f, 0x0d, 0x98, 0xa5, 0x17, 0x53, 0x3d, 0x47, 0x55, 0x1f, 0xca, + 0x7e, 0x13, 0xca, 0x3b, 0xae, 0x77, 0xda, 0xea, 0xb9, 0x6d, 0x6c, 0x89, 0x0a, 0x3d, 0xd7, 0x3b, + 0xd5, 0x7b, 0x5d, 0x9d, 0xdc, 0x0b, 0xf7, 0x58, 0xc5, 0x05, 0x5c, 0x4a, 0xda, 0x3f, 0xb3, 0x80, + 0x21, 0x53, 0x37, 0x1f, 0x49, 0x5e, 0x97, 0xe1, 0x6f, 0x99, 0xe1, 0x5f, 0x87, 0x99, 0x6e, 0xe0, + 0x0f, 0x07, 0x1b, 0xfa, 0x59, 0x68, 0x12, 0xe5, 0x7b, 0xf4, 0xa9, 0x41, 0x56, 0x6f, 0x92, 0xf8, + 0xda, 0xcf, 0xe5, 0x97, 0x16, 0x5c, 0x31, 0x8c, 0x68, 0x0d, 0xfb, 0x7d, 0x27, 0x18, 0xfd, 0x6f, + 0x6c, 0xf9, 0x83, 0x05, 0x97, 0x52, 0x17, 0x92, 0xbc, 0x5b, 0x11, 0x46, 0x6e, 0x1f, 0x31, 0x91, + 0x2c, 0x29, 0xf1, 0x84, 0x91, 0x2e, 0xe2, 0x65, 0xdd, 0x67, 0x14, 0xf1, 0xcf, 0xc3, 0x1c, 0x85, + 0x73, 0x2b, 0x16, 0x91, 0xa6, 0x8d, 0x71, 0xd9, 0x6a, 0xd2, 0x22, 0xe6, 0xc9, 0x83, 0x97, 0x53, + 0x25, 0xfc, 0x44, 0x83, 0xf8, 0x7d, 0xa8, 0x72, 0xe7, 0x27, 0x6f, 0xb9, 0x61, 0xe4, 0x77, 0x03, + 0xa7, 0x8f, 0x41, 0x72, 0x32, 0x6c, 0x9f, 0x0a, 0xd9, 0x47, 0xe4, 0xb9, 0xa2, 0xf0, 0xec, 0x6d, + 0xc3, 0x32, 0x49, 0xd8, 0x6f, 0x43, 0x49, 0x17, 0xc1, 0x19, 0x7d, 0xcd, 0x4b, 0xe9, 0xbe, 0x66, + 0x31, 0xdd, 0x4b, 0xbd, 0xbb, 0x83, 0xcd, 0x8b, 0xdb, 0xd6, 0x08, 0xf4, 0x1b, 0x0b, 0x2a, 0x86, + 0x89, 0x6c, 0x03, 0x16, 0x7a, 0x4e, 0x24, 0xbc, 0xf6, 0xe8, 0xf8, 0x81, 0x36, 0x4f, 0x45, 0x65, + 0xd2, 0x21, 0x99, 0xb6, 0xf3, 0x9a, 0x92, 0x4f, 0x4e, 0xf3, 0x3d, 0x28, 0x86, 0x22, 0x70, 0xd5, + 0xf3, 0x36, 0x51, 0x2b, 0xae, 0xdd, 0x95, 0x00, 0x1e, 0x5c, 0xe2, 0x85, 0xba, 0x58, 0x45, 0xd9, + 0x7f, 0x4d, 0x47, 0xb7, 0x0a, 0xac, 0xc9, 0x96, 0xeb, 0x11, 0xde, 0x9a, 0xce, 0xf4, 0x56, 0x62, + 0x5f, 0xee, 0x51, 0xf6, 0xd5, 0x20, 0x37, 0xb8, 0x7d, 0x5b, 0x35, 0x2c, 0x38, 0x94, 0x9c, 0x5b, + 0x14, 0x78, 0xc4, 0xb9, 0x25, 0x39, 0xeb, 0xaa, 0x4a, 0xc7, 0x21, 0x71, 0x6e, 0xad, 0xab, 0x72, + 0x1c, 0x87, 0xf6, 0xfb, 0xd0, 0xc8, 0x7a, 0x27, 0x2a, 0x44, 0x6f, 0x43, 0x39, 0x24, 0x96, 0x2b, + 0x26, 0x21, 0x20, 0x63, 0x5d, 0x22, 0x6d, 0xff, 0xd6, 0x82, 0xd9, 0x94, 0x63, 0x53, 0xd9, 0xa7, + 0xa0, 0xb2, 0x4f, 0x15, 0x2c, 0x8f, 0x2e, 0x23, 0xc7, 0x2d, 0x0f, 0xa9, 0xfb, 0x74, 0xdf, 0x16, + 0xb7, 0xee, 0x23, 0x25, 0x1b, 0x95, 0x32, 0xb7, 0x42, 0xa4, 0x4e, 0xe8, 0x70, 0x25, 0x6e, 0x9d, + 0x20, 0xd5, 0x51, 0x07, 0xb3, 0x3a, 0xd4, 0x21, 0x46, 0x4e, 0x34, 0x94, 0xf5, 0x51, 0x81, 0x2b, + 0x0a, 0x77, 0x3c, 0x75, 0xbd, 0x0e, 0x55, 0x44, 0x05, 0x4e, 0x63, 0x5b, 0xc8, 0xcf, 0x8d, 0xca, + 0x70, 0x84, 0x59, 0x2c, 0x77, 0x02, 0x11, 0x0e, 0x7b, 0xd1, 0x61, 0x92, 0x1c, 0x0d, 0x0e, 0x96, + 0x17, 0x92, 0x52, 0x61, 0xd3, 0xc8, 0x7c, 0x43, 0x24, 0xc1, 0x95, 0x24, 0xa2, 0xe0, 0xc2, 0xc4, + 0x2c, 0x86, 0x49, 0xcf, 0x39, 0x11, 0x3d, 0xa3, 0x3e, 0x48, 0x18, 0x68, 0x07, 0x11, 0x47, 0x46, + 0x3e, 0x36, 0x38, 0x6c, 0x0d, 0xa6, 0x23, 0x1d, 0x1a, 0xd7, 0xce, 0xb7, 0xe1, 0xc0, 0x77, 0xbd, + 0x88, 0x4f, 0x47, 0x21, 0xbe, 0xa1, 0xc5, 0xec, 0x69, 0x72, 0x86, 0xab, 0x8c, 0x98, 0xe5, 0x34, + 0xc6, 0xe8, 0x38, 0x73, 0x7a, 0xb4, 0xb1, 0xc5, 0x71, 0x88, 0x3d, 0x9f, 0x78, 0x28, 0xfa, 0x83, + 0x9e, 0x13, 0x1c, 0xaa, 0xef, 0x43, 0x39, 0xfa, 0x12, 0x3f, 0xce, 0x66, 0x2f, 0x40, 0x4d, 0xb3, + 0xf4, 0xf7, 0x62, 0x15, 0x9c, 0x13, 0x7c, 0xfb, 0x2f, 0x39, 0x58, 0xa0, 0x6f, 0xbf, 0xdc, 0xf1, + 0xba, 0xe2, 0x62, 0x50, 0x8e, 0x41, 0x56, 0x01, 0x4d, 0x0a, 0x64, 0xe5, 0xd3, 0xa4, 0x6f, 0xcb, + 0x58, 0xc6, 0x46, 0x62, 0xa0, 0xf6, 0xa4, 0x31, 0x02, 0x7a, 0xf8, 0xc0, 0x09, 0x3a, 0xdb, 0x9b, + 0x0a, 0x8e, 0x35, 0x89, 0x37, 0x4d, 0x43, 0xf9, 0x18, 0x65, 0xe5, 0x6d, 0x70, 0xd2, 0xbf, 0x11, + 0xcc, 0x8c, 0xfd, 0x46, 0x60, 0x36, 0x0d, 0xa5, 0x0b, 0x9a, 0x86, 0xf2, 0x23, 0x9b, 0x06, 0xc8, + 0x6a, 0x1a, 0x8c, 0x52, 0xbd, 0x92, 0x2e, 0xd5, 0xcd, 0x76, 0xa2, 0x3a, 0xd6, 0x4e, 0xe8, 0x32, + 0x7e, 0xf6, 0xdc, 0x32, 0x7e, 0xee, 0x6b, 0x95, 0xf1, 0xf3, 0x8f, 0x5d, 0xc6, 0x87, 0xc0, 0x4c, + 0x67, 0x2a, 0xe4, 0x78, 0x31, 0x86, 0x32, 0x09, 0x1b, 0x97, 0x12, 0xb4, 0x77, 0xfb, 0xa2, 0x45, + 0x53, 0x31, 0x98, 0x3d, 0xfe, 0x87, 0xcc, 0x3b, 0x50, 0x6c, 0x39, 0xfd, 0x41, 0x4f, 0xb0, 0xff, + 0x83, 0x2a, 0x06, 0x6f, 0x18, 0x39, 0xfd, 0xc1, 0x71, 0x3f, 0x54, 0x60, 0x52, 0x89, 0x79, 0xf2, + 0x57, 0x0b, 0x99, 0x78, 0x2c, 0x8a, 0x6c, 0x95, 0x60, 0x3e, 0xb6, 0x00, 0x12, 0x5b, 0xd8, 0x6d, + 0x28, 0xd2, 0x53, 0x9b, 0xc4, 0xb9, 0xc9, 0x2f, 0x3c, 0xea, 0xf7, 0x15, 0xb5, 0x80, 0xad, 0xc1, + 0x4c, 0x48, 0xc6, 0xe8, 0xbc, 0x32, 0x9f, 0x98, 0x4f, 0x7c, 0x25, 0xaf, 0xa5, 0xd8, 0x35, 0xa8, + 0x0c, 0x02, 0xbf, 0x7f, 0xac, 0x36, 0x94, 0x1f, 0x4a, 0x01, 0x59, 0x3b, 0xc4, 0x79, 0xe1, 0x43, + 0x98, 0x1f, 0x2b, 0x5f, 0x59, 0x15, 0x4a, 0x7b, 0xfb, 0xc7, 0x5b, 0x9c, 0xef, 0xf3, 0xda, 0x14, + 0xbb, 0x04, 0xf3, 0xbb, 0x77, 0x3e, 0x38, 0xde, 0xd9, 0x3e, 0xda, 0x3a, 0x3e, 0xe4, 0x77, 0xee, + 0x6e, 0xb5, 0x6a, 0x16, 0x32, 0x69, 0x7c, 0x7c, 0xb8, 0xbf, 0x7f, 0xbc, 0x73, 0x87, 0xdf, 0xdb, + 0xaa, 0x4d, 0xb3, 0x05, 0x98, 0x7d, 0x6f, 0xef, 0x9d, 0xbd, 0xfd, 0xf7, 0xf7, 0xd4, 0xe2, 0x5c, + 0xf3, 0x57, 0x16, 0x14, 0x51, 0xbd, 0x08, 0xd8, 0x0f, 0xa1, 0x1c, 0x17, 0xc1, 0xec, 0x4a, 0xaa, + 0x76, 0x36, 0x0b, 0xe3, 0xc6, 0x53, 0xa9, 0x29, 0xed, 0x65, 0x7b, 0x8a, 0xdd, 0x81, 0x4a, 0x2c, + 0x7c, 0xd4, 0xfc, 0x4f, 0x54, 0x34, 0xff, 0x69, 0x41, 0x4d, 0x39, 0xf8, 0x9e, 0xf0, 0x44, 0xe0, + 0x44, 0x7e, 0x6c, 0x18, 0x55, 0xb0, 0x63, 0x5a, 0xcd, 0x72, 0xf8, 0x7c, 0xc3, 0xb6, 0x01, 0xee, + 0x89, 0x48, 0x57, 0x0f, 0x57, 0xb3, 0xe1, 0x52, 0xea, 0x78, 0xe6, 0x1c, 0x2c, 0xd5, 0xaa, 0xee, + 0x01, 0x24, 0x11, 0xce, 0x12, 0xf4, 0x9f, 0xc0, 0xb0, 0xc6, 0xd5, 0xcc, 0xb9, 0xf8, 0xa4, 0xbf, + 0xcf, 0xc3, 0x0c, 0x4e, 0xb8, 0x22, 0x60, 0x6f, 0xc1, 0xec, 0x8f, 0x5c, 0xaf, 0x13, 0xff, 0xf8, + 0xc7, 0x32, 0x7e, 0x2d, 0xd4, 0x6a, 0x1b, 0x59, 0x53, 0x86, 0x0b, 0xaa, 0xfa, 0xe7, 0x84, 0xb6, + 0xf0, 0x22, 0x76, 0xce, 0x6f, 0x58, 0x8d, 0xa7, 0x27, 0xf8, 0xb1, 0x8a, 0x2d, 0xa8, 0x18, 0xbf, + 0x8f, 0x99, 0xb7, 0x35, 0xf1, 0xab, 0xd9, 0x45, 0x6a, 0xee, 0x01, 0x24, 0x3d, 0x35, 0xbb, 0xe0, + 0xeb, 0x5a, 0xe3, 0x6a, 0xe6, 0x5c, 0xac, 0xe8, 0x1d, 0x7d, 0x24, 0xd9, 0x9c, 0x5f, 0xa8, 0xea, + 0xd9, 0xcc, 0x66, 0xdf, 0x50, 0x76, 0x04, 0xf3, 0x63, 0xbd, 0x2c, 0x7b, 0xd4, 0x27, 0xa2, 0xc6, + 0xf2, 0xf9, 0x02, 0xb1, 0xde, 0x1f, 0x1b, 0x5f, 0x10, 0x74, 0x8f, 0xfc, 0x68, 0xcd, 0xf6, 0x79, + 0x02, 0xa6, 0xcd, 0xcd, 0x7f, 0xe5, 0xa0, 0xd6, 0x8a, 0x02, 0xe1, 0xf4, 0x5d, 0xaf, 0xab, 0x43, + 0xe6, 0x0d, 0x28, 0xaa, 0x1c, 0xf1, 0xb8, 0x2e, 0x5e, 0xb7, 0xf0, 0x3d, 0x3c, 0x11, 0xdf, 0xac, + 0x5b, 0x6c, 0xf7, 0x09, 0x7a, 0x67, 0xdd, 0x62, 0x1f, 0x7c, 0x33, 0xfe, 0x59, 0xb7, 0xd8, 0x87, + 0xdf, 0x9c, 0x87, 0xd6, 0x2d, 0x76, 0x00, 0x0b, 0x0a, 0x2b, 0x9e, 0x08, 0x3a, 0xac, 0x5b, 0xcd, + 0x3f, 0x5a, 0x30, 0xa3, 0x11, 0xeb, 0x38, 0xb3, 0xcf, 0xb0, 0x2f, 0xaa, 0xbe, 0xd5, 0x36, 0xcf, + 0x5d, 0x28, 0xf3, 0xc4, 0x51, 0x6d, 0xa3, 0xfe, 0xc9, 0x97, 0x4b, 0xd6, 0x67, 0x5f, 0x2e, 0x59, + 0xff, 0xf8, 0x72, 0xc9, 0xfa, 0xf5, 0x57, 0x4b, 0x53, 0x9f, 0x7d, 0xb5, 0x34, 0xf5, 0xf9, 0x57, + 0x4b, 0x53, 0x27, 0x45, 0xfa, 0xe7, 0x8e, 0x57, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x35, 0x3c, + 0x12, 0xda, 0x5d, 0x22, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6399,6 +6467,61 @@ func (m *QueryRangeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.DedicatedColumns) > 0 { + for iNdEx := len(m.DedicatedColumns) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.DedicatedColumns[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTempo(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x7a + } + } + if m.FooterSize != 0 { + i = encodeVarintTempo(dAtA, i, uint64(m.FooterSize)) + i-- + dAtA[i] = 0x70 + } + if m.Size_ != 0 { + i = encodeVarintTempo(dAtA, i, uint64(m.Size_)) + i-- + dAtA[i] = 0x68 + } + if len(m.Encoding) > 0 { + i -= len(m.Encoding) + copy(dAtA[i:], m.Encoding) + i = encodeVarintTempo(dAtA, i, uint64(len(m.Encoding))) + i-- + dAtA[i] = 0x62 + } + if len(m.Version) > 0 { + i -= len(m.Version) + copy(dAtA[i:], m.Version) + i = encodeVarintTempo(dAtA, i, uint64(len(m.Version))) + i-- + dAtA[i] = 0x5a + } + if m.PagesToSearch != 0 { + i = encodeVarintTempo(dAtA, i, uint64(m.PagesToSearch)) + i-- + dAtA[i] = 0x50 + } + if m.StartPage != 0 { + i = encodeVarintTempo(dAtA, i, uint64(m.StartPage)) + i-- + dAtA[i] = 0x48 + } + if len(m.BlockID) > 0 { + i -= len(m.BlockID) + copy(dAtA[i:], m.BlockID) + i = encodeVarintTempo(dAtA, i, uint64(len(m.BlockID))) + i-- + dAtA[i] = 0x42 + } if len(m.QueryMode) > 0 { i -= len(m.QueryMode) copy(dAtA[i:], m.QueryMode) @@ -7576,6 +7699,36 @@ func (m *QueryRangeRequest) Size() (n int) { if l > 0 { n += 1 + l + sovTempo(uint64(l)) } + l = len(m.BlockID) + if l > 0 { + n += 1 + l + sovTempo(uint64(l)) + } + if m.StartPage != 0 { + n += 1 + sovTempo(uint64(m.StartPage)) + } + if m.PagesToSearch != 0 { + n += 1 + sovTempo(uint64(m.PagesToSearch)) + } + l = len(m.Version) + if l > 0 { + n += 1 + l + sovTempo(uint64(l)) + } + l = len(m.Encoding) + if l > 0 { + n += 1 + l + sovTempo(uint64(l)) + } + if m.Size_ != 0 { + n += 1 + sovTempo(uint64(m.Size_)) + } + if m.FooterSize != 0 { + n += 1 + sovTempo(uint64(m.FooterSize)) + } + if len(m.DedicatedColumns) > 0 { + for _, e := range m.DedicatedColumns { + l = e.Size() + n += 1 + l + sovTempo(uint64(l)) + } + } return n } @@ -13964,6 +14117,212 @@ func (m *QueryRangeRequest) Unmarshal(dAtA []byte) error { } m.QueryMode = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTempo + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTempo + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BlockID = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartPage", wireType) + } + m.StartPage = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartPage |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PagesToSearch", wireType) + } + m.PagesToSearch = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PagesToSearch |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTempo + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTempo + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Encoding", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTempo + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTempo + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Encoding = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Size_", wireType) + } + m.Size_ = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Size_ |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FooterSize", wireType) + } + m.FooterSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FooterSize |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DedicatedColumns", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTempo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTempo + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTempo + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DedicatedColumns = append(m.DedicatedColumns, &DedicatedColumn{}) + if err := m.DedicatedColumns[len(m.DedicatedColumns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTempo(dAtA[iNdEx:]) diff --git a/pkg/tempopb/tempo.proto b/pkg/tempopb/tempo.proto index 466ebd30ff2..253ea1b8b89 100644 --- a/pkg/tempopb/tempo.proto +++ b/pkg/tempopb/tempo.proto @@ -367,6 +367,15 @@ message QueryRangeRequest { uint32 shardID = 5; uint32 shardCount = 6; string queryMode = 7; + // New RF1 fields + string blockID = 8; + uint32 startPage = 9; + uint32 pagesToSearch = 10; + string version = 11; + string encoding = 12; + uint64 size = 13; // total size of data file + uint32 footerSize = 14; // size of file footer (parquet) + repeated DedicatedColumn dedicatedColumns = 15; } message QueryRangeResponse { diff --git a/pkg/traceql/engine_metrics.go b/pkg/traceql/engine_metrics.go index f06022fc471..296ed0701a1 100644 --- a/pkg/traceql/engine_metrics.go +++ b/pkg/traceql/engine_metrics.go @@ -628,6 +628,10 @@ func optimize(req *FetchSpansRequest) { req.SecondPassConditions = nil } } + + if len(req.SecondPassConditions) == 0 { + req.SecondPass = nil + } } func lookup(needles []Attribute, haystack Span) Static { diff --git a/tempodb/backend/block_meta.go b/tempodb/backend/block_meta.go index ac68c113956..d94b8226ca0 100644 --- a/tempodb/backend/block_meta.go +++ b/tempodb/backend/block_meta.go @@ -89,6 +89,11 @@ type CompactedBlockMeta struct { CompactedTime time.Time `json:"compactedTime"` } +const ( + DefaultReplicationFactor = 0 // Replication factor for blocks from the ingester. This is the default value to indicate RF3. + MetricsGeneratorReplicationFactor = 1 +) + // The BlockMeta data that is stored for each individual block. type BlockMeta struct { // A Version that indicates the block format. This includes specifics of how the indexes and data is stored. diff --git a/tempodb/encoding/common/interfaces.go b/tempodb/encoding/common/interfaces.go index 993737f4f7f..67e8477ad7b 100644 --- a/tempodb/encoding/common/interfaces.go +++ b/tempodb/encoding/common/interfaces.go @@ -30,35 +30,34 @@ type Searcher interface { } type SearchOptions struct { - ChunkSizeBytes uint32 // Buffer size to read from backend storage. - StartPage int // Controls searching only a subset of the block. Which page to begin searching at. - TotalPages int // Controls searching only a subset of the block. How many pages to search. - MaxBytes int // Max allowable trace size in bytes. Traces exceeding this are not searched. - PrefetchTraceCount int // How many traces to prefetch async. - ReadBufferCount int - ReadBufferSize int + ChunkSizeBytes uint32 // Buffer size to read from backend storage. + StartPage int // Controls searching only a subset of the block. Which page to begin searching at. + TotalPages int // Controls searching only a subset of the block. How many pages to search. + MaxBytes int // Max allowable trace size in bytes. Traces exceeding this are not searched. + PrefetchTraceCount int // How many traces to prefetch async. + ReadBufferCount int + ReadBufferSize int + BlockReplicationFactor int // Only blocks with this replication factor will be searched. Set to 1 to search generator blocks (RF=1). } -// DefaultSearchOptions() is used in a lot of places such as local ingester searches. It is important +// DefaultSearchOptions is used in a lot of places such as local ingester searches. It is important // in these cases to set a reasonable read buffer size and count to prevent constant tiny readranges // against the local backend. // TODO: Note that there is another method of creating "default search options" that looks like this: // tempodb.SearchConfig{}.ApplyToOptions(&searchOpts). we should consolidate these. func DefaultSearchOptions() SearchOptions { return SearchOptions{ - ReadBufferCount: 32, - ReadBufferSize: 1024 * 1024, - ChunkSizeBytes: 4 * 1024 * 1024, + ReadBufferCount: 32, + ReadBufferSize: 1024 * 1024, + ChunkSizeBytes: 4 * 1024 * 1024, + BlockReplicationFactor: backend.DefaultReplicationFactor, } } func DefaultSearchOptionsWithMaxBytes(maxBytes int) SearchOptions { - return SearchOptions{ - ReadBufferCount: 32, - ReadBufferSize: 1024 * 1024, - ChunkSizeBytes: 4 * 1024 * 1024, - MaxBytes: maxBytes, - } + opts := DefaultSearchOptions() + opts.MaxBytes = maxBytes + return opts } type Compactor interface { diff --git a/tempodb/tempodb.go b/tempodb/tempodb.go index ba746ad5437..ac488fc27e2 100644 --- a/tempodb/tempodb.go +++ b/tempodb/tempodb.go @@ -302,13 +302,13 @@ func (rw *readerWriter) Find(ctx context.Context, tenantID string, id common.ID, compactedBlocksSearched := 0 for _, b := range blocklist { - if includeBlock(b, id, blockStartBytes, blockEndBytes, timeStart, timeEnd) { + if includeBlock(b, id, blockStartBytes, blockEndBytes, timeStart, timeEnd, opts.BlockReplicationFactor) { copiedBlocklist = append(copiedBlocklist, b) blocksSearched++ } } for _, c := range compactedBlocklist { - if includeCompactedBlock(c, id, blockStartBytes, blockEndBytes, rw.cfg.BlocklistPoll, timeStart, timeEnd) { + if includeCompactedBlock(c, id, blockStartBytes, blockEndBytes, rw.cfg.BlocklistPoll, timeStart, timeEnd, opts.BlockReplicationFactor) { copiedBlocklist = append(copiedBlocklist, &c.BlockMeta) compactedBlocksSearched++ } @@ -552,7 +552,7 @@ func (rw *readerWriter) pollBlocklist() { } // includeBlock indicates whether a given block should be included in a backend search -func includeBlock(b *backend.BlockMeta, _ common.ID, blockStart []byte, blockEnd []byte, timeStart int64, timeEnd int64) bool { +func includeBlock(b *backend.BlockMeta, _ common.ID, blockStart, blockEnd []byte, timeStart, timeEnd int64, replicationFactor int) bool { // todo: restore this functionality once it works. min/max ids are currently not recorded // https://github.com/grafana/tempo/issues/1903 // correctly in a block @@ -573,16 +573,16 @@ func includeBlock(b *backend.BlockMeta, _ common.ID, blockStart []byte, blockEnd return false } - return true + return b.ReplicationFactor == uint32(replicationFactor) } // if block is compacted within lookback period, and is within shard ranges, include it in search -func includeCompactedBlock(c *backend.CompactedBlockMeta, id common.ID, blockStart []byte, blockEnd []byte, poll time.Duration, timeStart int64, timeEnd int64) bool { +func includeCompactedBlock(c *backend.CompactedBlockMeta, id common.ID, blockStart, blockEnd []byte, poll time.Duration, timeStart, timeEnd int64, replicationFactor int) bool { lookback := time.Now().Add(-(2 * poll)) if c.CompactedTime.Before(lookback) { return false } - return includeBlock(&c.BlockMeta, id, blockStart, blockEnd, timeStart, timeEnd) + return includeBlock(&c.BlockMeta, id, blockStart, blockEnd, timeStart, timeEnd, replicationFactor) } // createLegacyCache uses the config to return a cache and a list of roles. diff --git a/tempodb/tempodb_test.go b/tempodb/tempodb_test.go index 44f4b1b3921..54758e9ca30 100644 --- a/tempodb/tempodb_test.go +++ b/tempodb/tempodb_test.go @@ -440,7 +440,7 @@ func TestIncludeBlock(t *testing.T) { e, err := tc.blockEnd.MarshalBinary() require.NoError(t, err) - assert.Equal(t, tc.expected, includeBlock(tc.meta, tc.searchID, s, e, tc.start, tc.end)) + assert.Equal(t, tc.expected, includeBlock(tc.meta, tc.searchID, s, e, tc.start, tc.end, 0)) }) } } @@ -517,7 +517,7 @@ func TestIncludeCompactedBlock(t *testing.T) { e, err := tc.blockEnd.MarshalBinary() require.NoError(t, err) - assert.Equal(t, tc.expected, includeCompactedBlock(tc.meta, tc.searchID, s, e, blocklistPoll, tc.start, tc.end)) + assert.Equal(t, tc.expected, includeCompactedBlock(tc.meta, tc.searchID, s, e, blocklistPoll, tc.start, tc.end, 0)) }) } }