diff --git a/integration/querier_test.go b/integration/querier_test.go index 7e16b587dbb..a39f47dd277 100644 --- a/integration/querier_test.go +++ b/integration/querier_test.go @@ -1375,3 +1375,109 @@ func TestQuerierEngineConfigs(t *testing.T) { } } + +func TestQuerierDistributedExecution(t *testing.T) { + // e2e test setup + s, err := e2e.NewScenario(networkName) + require.NoError(t, err) + defer s.Close() + + consul := e2edb.NewConsulWithName("consul") + memcached := e2ecache.NewMemcached() + require.NoError(t, s.StartAndWaitReady(consul, memcached)) + + // initialize the flags + baseFlags := mergeFlags(AlertmanagerLocalFlags(), BlocksStorageFlags()) + flags := mergeFlags( + baseFlags, + map[string]string{ + "-blocks-storage.tsdb.head-compaction-interval": "4m", + "-blocks-storage.tsdb.block-ranges-period": "2h", + "-blocks-storage.tsdb.ship-interval": "1h", + "-blocks-storage.bucket-store.sync-interval": "1s", + "-blocks-storage.tsdb.retention-period": "24h", + "-blocks-storage.bucket-store.index-cache.backend": tsdb.IndexCacheBackendInMemory, + "-querier.query-store-for-labels-enabled": "true", + // Ingester. + "-ring.store": "consul", + "-consul.hostname": consul.NetworkHTTPEndpoint(), + // Distributor. + "-distributor.replication-factor": "1", + // Store-gateway. + "-store-gateway.sharding-enabled": "false", + // Alert manager + "-alertmanager.web.external-url": "http://localhost/alertmanager", + "-frontend.query-vertical-shard-size": "1", + "-frontend.max-cache-freshness": "1m", + // enable experimental promQL funcs + "-querier.enable-promql-experimental-functions": "true", + // enable distributed execution (logical plan execution) + "-querier.distributed-exec-enabled": "true", + }, + ) + + minio := e2edb.NewMinio(9000, flags["-blocks-storage.s3.bucket-name"]) + require.NoError(t, s.StartAndWaitReady(minio)) + + // start services + queryScheduler := e2ecortex.NewQueryScheduler("query-scheduler", flags, "") + require.NoError(t, s.StartAndWaitReady(queryScheduler)) + flags["-frontend.scheduler-address"] = queryScheduler.NetworkGRPCEndpoint() + flags["-querier.scheduler-address"] = queryScheduler.NetworkGRPCEndpoint() + + queryFrontend := e2ecortex.NewQueryFrontend("query-frontend", flags, "") + require.NoError(t, s.Start(queryFrontend)) + + ingester := e2ecortex.NewIngester("ingester", e2ecortex.RingStoreConsul, consul.NetworkHTTPEndpoint(), flags, "") + distributor := e2ecortex.NewDistributor("distributor", e2ecortex.RingStoreConsul, consul.NetworkHTTPEndpoint(), flags, "") + querier1 := e2ecortex.NewQuerier("querier-1", e2ecortex.RingStoreConsul, consul.NetworkHTTPEndpoint(), flags, "") + querier2 := e2ecortex.NewQuerier("querier-2", e2ecortex.RingStoreConsul, consul.NetworkHTTPEndpoint(), flags, "") + + require.NoError(t, s.StartAndWaitReady(querier1, querier2, ingester, distributor)) + require.NoError(t, s.WaitReady(queryFrontend)) + + // wait until distributor and queriers have updated the ring. + require.NoError(t, distributor.WaitSumMetrics(e2e.Equals(512), "cortex_ring_tokens_total")) + require.NoError(t, querier1.WaitSumMetrics(e2e.Equals(512), "cortex_ring_tokens_total")) + require.NoError(t, querier2.WaitSumMetrics(e2e.Equals(512), "cortex_ring_tokens_total")) + + // push some series to Cortex. + distClient, err := e2ecortex.NewClient(distributor.HTTPEndpoint(), "", "", "", userID) + require.NoError(t, err) + + series1Timestamp := time.Now() + series2Timestamp := series1Timestamp.Add(time.Minute * 1) + series1, expectedVector1 := generateSeries("series_1", series1Timestamp, prompb.Label{Name: "series_1", Value: "series_1"}) + series2, expectedVector2 := generateSeries("series_2", series2Timestamp, prompb.Label{Name: "series_2", Value: "series_2"}) + + res, err := distClient.Push(series1) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + + res, err = distClient.Push(series2) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + + for _, q := range []*e2ecortex.CortexService{querier1, querier2} { + c, err := e2ecortex.NewClient("", q.HTTPEndpoint(), "", "", userID) + require.NoError(t, err) + + _, err = c.Query("series_1", series1Timestamp) + require.NoError(t, err) + } + + require.NoError(t, queryScheduler.WaitSumMetrics(e2e.Equals(2), "cortex_query_scheduler_connected_querier_clients")) + + // main tests + // - make sure queries are still executable with distributed execution enabled + var body []byte + res, body, err = distClient.QueryRaw(`sum({job="test"})`, series1Timestamp, map[string]string{}) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + require.Equal(t, expectedVector1, string(body)) + + res, body, err = distClient.QueryRaw(`sum({job="test"})`, series2Timestamp, map[string]string{}) + require.NoError(t, err) + require.Equal(t, 200, res.StatusCode) + require.Equal(t, expectedVector2, string(body)) +} diff --git a/pkg/api/api.go b/pkg/api/api.go index 1c68c426d8b..0aa73887633 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -3,6 +3,8 @@ package api import ( "context" "flag" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution/querierpb" "net/http" "path" "strings" @@ -480,6 +482,10 @@ func (a *API) RegisterQueryScheduler(f *scheduler.Scheduler) { schedulerpb.RegisterSchedulerForQuerierServer(a.server.GRPC, f) } +func (a *API) RegisterQuerierServer(q *distributed_execution.QuerierServer) { + querierpb.RegisterQuerierServer(a.server.GRPC, q) +} + // RegisterServiceMapHandler registers the Cortex structs service handler // TODO: Refactor this code to be accomplished using the services.ServiceManager // or a future module manager #2291 diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 9bcc6a6906e..0084b696a28 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -3,6 +3,7 @@ package api import ( "context" "encoding/json" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "html/template" "net/http" "path" @@ -19,13 +20,13 @@ import ( "github.com/prometheus/common/route" "github.com/prometheus/common/version" "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" v1 "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" "github.com/cortexproject/cortex/pkg/api/queryapi" + "github.com/cortexproject/cortex/pkg/engine" "github.com/cortexproject/cortex/pkg/querier" "github.com/cortexproject/cortex/pkg/querier/codec" "github.com/cortexproject/cortex/pkg/querier/stats" @@ -163,10 +164,12 @@ func NewQuerierHandler( cfg Config, queryable storage.SampleAndChunkQueryable, exemplarQueryable storage.ExemplarQueryable, - engine promql.QueryEngine, + engine engine.QueryEngine, + queryResultCache *distributed_execution.QueryResultCache, metadataQuerier querier.MetadataQuerier, reg prometheus.Registerer, logger log.Logger, + distributedExecEnabled bool, ) http.Handler { // Prometheus histograms for requests to the querier. querierRequestDuration := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ @@ -280,7 +283,7 @@ func NewQuerierHandler( legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) api.Register(legacyPromRouter) - queryAPI := queryapi.NewQueryAPI(engine, translateSampleAndChunkQueryable, statsRenderer, logger, codecs, corsOrigin) + queryAPI := queryapi.NewQueryAPI(engine, queryResultCache, translateSampleAndChunkQueryable, statsRenderer, logger, codecs, corsOrigin, distributedExecEnabled) // TODO(gotjosh): This custom handler is temporary until we're able to vendor the changes in: // https://github.com/prometheus/prometheus/pull/7125/files diff --git a/pkg/api/handlers_test.go b/pkg/api/handlers_test.go index 32e84d70a97..488d628f4a9 100644 --- a/pkg/api/handlers_test.go +++ b/pkg/api/handlers_test.go @@ -232,7 +232,7 @@ func TestBuildInfoAPI(t *testing.T) { version.Version = tc.version version.Branch = tc.branch version.Revision = tc.revision - handler := NewQuerierHandler(cfg, nil, nil, nil, nil, nil, &FakeLogger{}) + handler := NewQuerierHandler(cfg, nil, nil, nil, nil, nil, nil, &FakeLogger{}, false) writer := httptest.NewRecorder() req := httptest.NewRequest("GET", "/api/v1/status/buildinfo", nil) req = req.WithContext(user.InjectOrgID(req.Context(), "test")) diff --git a/pkg/api/queryapi/query_api.go b/pkg/api/queryapi/query_api.go index 5dd125a6c39..41de13bd305 100644 --- a/pkg/api/queryapi/query_api.go +++ b/pkg/api/queryapi/query_api.go @@ -3,6 +3,7 @@ package queryapi import ( "context" "fmt" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "net/http" "strconv" "time" @@ -25,31 +26,37 @@ import ( ) type QueryAPI struct { - queryable storage.SampleAndChunkQueryable - queryEngine promql.QueryEngine - now func() time.Time - statsRenderer v1.StatsRenderer - logger log.Logger - codecs []v1.Codec - CORSOrigin *regexp.Regexp + queryable storage.SampleAndChunkQueryable + queryEngine engine.QueryEngine + queryResultCache *distributed_execution.QueryResultCache + now func() time.Time + statsRenderer v1.StatsRenderer + logger log.Logger + codecs []v1.Codec + CORSOrigin *regexp.Regexp + distributedExecEnabled bool } func NewQueryAPI( - qe promql.QueryEngine, + qe engine.QueryEngine, + queryResultCache *distributed_execution.QueryResultCache, q storage.SampleAndChunkQueryable, statsRenderer v1.StatsRenderer, logger log.Logger, codecs []v1.Codec, CORSOrigin *regexp.Regexp, + distributedExecEnabled bool, ) *QueryAPI { return &QueryAPI{ - queryEngine: qe, - queryable: q, - statsRenderer: statsRenderer, - logger: logger, - codecs: codecs, - CORSOrigin: CORSOrigin, - now: time.Now, + queryEngine: qe, + queryResultCache: queryResultCache, + queryable: q, + statsRenderer: statsRenderer, + logger: logger, + codecs: codecs, + CORSOrigin: CORSOrigin, + now: time.Now, + distributedExecEnabled: distributedExecEnabled, } } @@ -101,10 +108,29 @@ func (q *QueryAPI) RangeQueryHandler(r *http.Request) (result apiFuncResult) { ctx = engine.AddEngineTypeToContext(ctx, r) ctx = querier.AddBlockStoreTypeToContext(ctx, r.Header.Get(querier.BlockStoreTypeHeader)) - qry, err := q.queryEngine.NewRangeQuery(ctx, q.queryable, opts, r.FormValue("query"), convertMsToTime(start), convertMsToTime(end), convertMsToDuration(step)) - if err != nil { - return invalidParamError(httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error()), "query") + + var qry promql.Query + startTime := convertMsToTime(start) + endTime := convertMsToTime(end) + stepDuration := convertMsToDuration(step) + + byteLP := []byte(r.PostFormValue("plan")) + if len(byteLP) != 0 { + logicalPlan, err := distributed_execution.Unmarshal(byteLP) + if err != nil { + return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("invalid logical plan: %v", err)}, nil, nil} + } + qry, err = q.queryEngine.MakeRangeQueryFromPlan(ctx, q.queryable, opts, logicalPlan, startTime, endTime, stepDuration, r.FormValue("query")) + if err != nil { + return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("failed to create range query from logical plan: %v", err)}, nil, nil} + } + } else { // if there is logical plan field is empty, fall back + qry, err = q.queryEngine.NewRangeQuery(ctx, q.queryable, opts, r.FormValue("query"), startTime, endTime, stepDuration) + if err != nil { + return invalidParamError(httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error()), "query") + } } + // From now on, we must only return with a finalizer in the result (to // be called by the caller) or call qry.Close ourselves (which is // required in the case of a panic). @@ -116,6 +142,14 @@ func (q *QueryAPI) RangeQueryHandler(r *http.Request) (result apiFuncResult) { ctx = httputil.ContextFromRequest(ctx, r) + if q.distributedExecEnabled { + isRoot, queryID, fragmentID, _, _ := distributed_execution.ExtractFragmentMetaData(ctx) + if !isRoot { + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + q.queryResultCache.InitWriting(key) + } + } + res := qry.Exec(ctx) if res.Err != nil { return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} @@ -157,9 +191,45 @@ func (q *QueryAPI) InstantQueryHandler(r *http.Request) (result apiFuncResult) { ctx = engine.AddEngineTypeToContext(ctx, r) ctx = querier.AddBlockStoreTypeToContext(ctx, r.Header.Get(querier.BlockStoreTypeHeader)) - qry, err := q.queryEngine.NewInstantQuery(ctx, q.queryable, opts, r.FormValue("query"), convertMsToTime(ts)) - if err != nil { - return invalidParamError(httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error()), "query") + + var isRoot bool + var queryID, fragmentID uint64 + if q.distributedExecEnabled { + isRoot, queryID, fragmentID, _, _ = distributed_execution.ExtractFragmentMetaData(ctx) + if !isRoot { + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + q.queryResultCache.InitWriting(key) + } + } + + var qry promql.Query + tsTime := convertMsToTime(ts) + + byteLP := []byte(r.PostFormValue("plan")) + if len(byteLP) != 0 { + logicalPlan, err := distributed_execution.Unmarshal(byteLP) + if err != nil { + if q.distributedExecEnabled { + if !isRoot { + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + q.queryResultCache.SetError(key) + } + } + return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("invalid logical plan: %v", err)}, nil, nil} + } + qry, err = q.queryEngine.MakeInstantQueryFromPlan(ctx, q.queryable, opts, logicalPlan, tsTime, r.FormValue("query")) + if err != nil { + if !isRoot { + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + q.queryResultCache.SetError(key) + } + return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("failed to create instant query from logical plan: %v", err)}, nil, nil} + } + } else { // if there is logical plan field is empty, fall back + qry, err = q.queryEngine.NewInstantQuery(ctx, q.queryable, opts, r.FormValue("query"), tsTime) + if err != nil { + return invalidParamError(httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error()), "query") + } } // From now on, we must only return with a finalizer in the result (to @@ -203,6 +273,20 @@ func (q *QueryAPI) Wrap(f apiFunc) http.HandlerFunc { } if result.data != nil { + ctx := httputil.ContextFromRequest(r.Context(), r) + + if q.distributedExecEnabled { + isRoot, queryID, fragmentID, _, _ := distributed_execution.ExtractFragmentMetaData(ctx) + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + + q.queryResultCache.SetComplete(key, result.data) + + if isRoot { + q.respond(w, r, result.data, result.warnings, r.FormValue("query")) + } + return + } + q.respond(w, r, result.data, result.warnings, r.FormValue("query")) return } @@ -240,7 +324,9 @@ func (q *QueryAPI) respond(w http.ResponseWriter, req *http.Request, data interf w.Header().Set("Content-Type", codec.ContentType().String()) w.Header().Set("X-Uncompressed-Length", strconv.Itoa(len(b))) w.WriteHeader(http.StatusOK) - if n, err := w.Write(b); err != nil { + + n, err := w.Write(b) + if err != nil { level.Error(q.logger).Log("error writing response", "url", req.URL, "bytesWritten", n, "err", err) } } diff --git a/pkg/api/queryapi/query_api_test.go b/pkg/api/queryapi/query_api_test.go index 028184a12b8..e6002846160 100644 --- a/pkg/api/queryapi/query_api_test.go +++ b/pkg/api/queryapi/query_api_test.go @@ -4,24 +4,32 @@ import ( "context" "errors" "fmt" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "io" "net/http" "net/http/httptest" + "net/url" + "strings" "testing" "time" "github.com/go-kit/log" "github.com/gorilla/mux" "github.com/grafana/regexp" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" v1 "github.com/prometheus/prometheus/web/api/v1" "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" "github.com/weaveworks/common/user" + engine2 "github.com/cortexproject/cortex/pkg/engine" "github.com/cortexproject/cortex/pkg/querier" "github.com/cortexproject/cortex/pkg/querier/series" "github.com/cortexproject/cortex/pkg/querier/stats" @@ -64,10 +72,14 @@ func (mockQuerier) Close() error { } func Test_CustomAPI(t *testing.T) { - engine := promql.NewEngine(promql.EngineOpts{ - MaxSamples: 100, - Timeout: time.Second * 2, - }) + engine := engine2.New( + promql.EngineOpts{ + MaxSamples: 100, + Timeout: time.Second * 2, + }, + engine2.ThanosEngineConfig{Enabled: false}, + prometheus.NewRegistry()) + mockQueryable := &mockSampleAndChunkQueryable{ queryableFn: func(_, _ int64) (storage.Querier, error) { return mockQuerier{ @@ -172,13 +184,13 @@ func Test_CustomAPI(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - c := NewQueryAPI(engine, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{v1.JSONCodec{}}, regexp.MustCompile(".*")) + c := NewQueryAPI(engine, nil, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{v1.JSONCodec{}}, regexp.MustCompile(".*"), false) router := mux.NewRouter() - router.Path("/api/v1/query").Methods("GET").Handler(c.Wrap(c.InstantQueryHandler)) - router.Path("/api/v1/query_range").Methods("GET").Handler(c.Wrap(c.RangeQueryHandler)) + router.Path("/api/v1/query").Methods("POST").Handler(c.Wrap(c.InstantQueryHandler)) + router.Path("/api/v1/query_range").Methods("POST").Handler(c.Wrap(c.RangeQueryHandler)) - req := httptest.NewRequest(http.MethodGet, test.path, nil) + req := httptest.NewRequest(http.MethodPost, test.path, nil) ctx := context.Background() _, ctx = stats.ContextWithEmptyStats(ctx) req = req.WithContext(user.InjectOrgID(ctx, "user1")) @@ -209,10 +221,14 @@ func (m *mockCodec) Encode(_ *v1.Response) ([]byte, error) { } func Test_InvalidCodec(t *testing.T) { - engine := promql.NewEngine(promql.EngineOpts{ - MaxSamples: 100, - Timeout: time.Second * 2, - }) + engine := engine2.New( + promql.EngineOpts{ + MaxSamples: 100, + Timeout: time.Second * 2, + }, + engine2.ThanosEngineConfig{Enabled: false}, + prometheus.NewRegistry()) + mockQueryable := &mockSampleAndChunkQueryable{ queryableFn: func(_, _ int64) (storage.Querier, error) { return mockQuerier{ @@ -229,11 +245,11 @@ func Test_InvalidCodec(t *testing.T) { }, } - queryAPI := NewQueryAPI(engine, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{&mockCodec{}}, regexp.MustCompile(".*")) + queryAPI := NewQueryAPI(engine, nil, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{&mockCodec{}}, regexp.MustCompile(".*"), false) router := mux.NewRouter() - router.Path("/api/v1/query").Methods("GET").Handler(queryAPI.Wrap(queryAPI.InstantQueryHandler)) + router.Path("/api/v1/query").Methods("POST").Handler(queryAPI.Wrap(queryAPI.InstantQueryHandler)) - req := httptest.NewRequest(http.MethodGet, "/api/v1/query?query=test", nil) + req := httptest.NewRequest(http.MethodPost, "/api/v1/query?query=test", nil) ctx := context.Background() _, ctx = stats.ContextWithEmptyStats(ctx) req = req.WithContext(user.InjectOrgID(ctx, "user1")) @@ -244,10 +260,14 @@ func Test_InvalidCodec(t *testing.T) { } func Test_CustomAPI_StatsRenderer(t *testing.T) { - engine := promql.NewEngine(promql.EngineOpts{ - MaxSamples: 100, - Timeout: time.Second * 2, - }) + engine := engine2.New( + promql.EngineOpts{ + MaxSamples: 100, + Timeout: time.Second * 2, + }, + engine2.ThanosEngineConfig{Enabled: false}, + prometheus.NewRegistry()) + mockQueryable := &mockSampleAndChunkQueryable{ queryableFn: func(_, _ int64) (storage.Querier, error) { return mockQuerier{ @@ -266,12 +286,12 @@ func Test_CustomAPI_StatsRenderer(t *testing.T) { }, } - queryAPI := NewQueryAPI(engine, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{v1.JSONCodec{}}, regexp.MustCompile(".*")) + queryAPI := NewQueryAPI(engine, nil, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{v1.JSONCodec{}}, regexp.MustCompile(".*"), false) router := mux.NewRouter() - router.Path("/api/v1/query_range").Methods("GET").Handler(queryAPI.Wrap(queryAPI.RangeQueryHandler)) + router.Path("/api/v1/query_range").Methods("POST").Handler(queryAPI.Wrap(queryAPI.RangeQueryHandler)) - req := httptest.NewRequest(http.MethodGet, "/api/v1/query_range?end=1536673680&query=test&start=1536673665&step=5", nil) + req := httptest.NewRequest(http.MethodPost, "/api/v1/query_range?end=1536673680&query=test&start=1536673665&step=5", nil) ctx := context.Background() _, ctx = stats.ContextWithEmptyStats(ctx) req = req.WithContext(user.InjectOrgID(ctx, "user1")) @@ -285,3 +305,201 @@ func Test_CustomAPI_StatsRenderer(t *testing.T) { require.Equal(t, uint64(4), queryStats.LoadPeakSamples()) require.Equal(t, uint64(4), queryStats.LoadScannedSamples()) } + +func Test_Logicalplan_Requests(t *testing.T) { + engine := engine2.New( + promql.EngineOpts{ + MaxSamples: 100, + Timeout: time.Second * 2, + }, + engine2.ThanosEngineConfig{Enabled: true}, + prometheus.NewRegistry(), + ) + + mockMatrix := model.Matrix{ + { + Metric: model.Metric{"__name__": "test", "foo": "bar"}, + Values: []model.SamplePair{ + {Timestamp: 1536673665000, Value: 0}, + {Timestamp: 1536673670000, Value: 1}, + }, + }, + } + + mockQueryable := &mockSampleAndChunkQueryable{ + queryableFn: func(_, _ int64) (storage.Querier, error) { + return mockQuerier{matrix: mockMatrix}, nil + }, + } + + tests := []struct { + name string + path string + start int64 + end int64 + stepDuration int64 + requestBody func(t *testing.T) []byte + expectedCode int + expectedBody string + }{ + { + name: "[Range Query] with valid logical plan and empty query string", + path: "/api/v1/query_range?end=1536673680&query=&start=1536673665&step=5", + start: 1536673665, + end: 1536673680, + stepDuration: 5, + requestBody: func(t *testing.T) []byte { + return createTestLogicalPlan(t, 1536673665, 1536673680, 5) + }, + expectedCode: http.StatusOK, + expectedBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"test","foo":"bar"},"values":[[1536673665,"0"],[1536673670,"1"],[1536673675,"1"],[1536673680,"1"]]}]}}`, + }, + { + name: "[Range Query] with corrupted logical plan", // will throw an error from unmarhsal step + path: "/api/v1/query_range?end=1536673680&query=test&start=1536673665&step=5", + start: 1536673665, + end: 1536673680, + stepDuration: 5, + requestBody: func(t *testing.T) []byte { + return append(createTestLogicalPlan(t, 1536673665, 1536673680, 5), []byte("random data")...) + }, + expectedCode: http.StatusInternalServerError, + expectedBody: `{"status":"error","errorType":"server_error","error":"invalid logical plan: invalid character 'r' after top-level value"}`, + }, + { + name: "[Range Query] with empty body and non-empty query string", // fall back to promql query execution + path: "/api/v1/query_range?end=1536673680&query=test&start=1536673665&step=5", + start: 1536673665, + end: 1536673680, + stepDuration: 5, + requestBody: func(t *testing.T) []byte { + return []byte{} + }, + expectedCode: http.StatusOK, + expectedBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"test","foo":"bar"},"values":[[1536673665,"0"],[1536673670,"1"],[1536673675,"1"],[1536673680,"1"]]}]}}`, + }, + { + name: "[Range Query] with empty body and empty query string", // fall back to promql query execution, but will have error because of empty query string + path: "/api/v1/query_range?end=1536673680&query=&start=1536673665&step=5", + start: 1536673665, + end: 1536673680, + stepDuration: 5, + requestBody: func(t *testing.T) []byte { + return []byte{} + }, + expectedCode: http.StatusBadRequest, + expectedBody: "{\"status\":\"error\",\"errorType\":\"bad_data\",\"error\":\"invalid parameter \\\"query\\\"; unknown position: parse error: no expression found in input\"}", + }, + { + name: "[Instant Query] with valid logical plan and empty query string", + path: "/api/v1/query?query=test&time=1536673670", + start: 1536673670, + end: 1536673670, + stepDuration: 0, + requestBody: func(t *testing.T) []byte { + return createTestLogicalPlan(t, 1536673670, 1536673670, 0) + }, + expectedCode: http.StatusOK, + expectedBody: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"test","foo":"bar"},"value":[1536673670,"1"]}]}}`, + }, + { + name: "[Instant Query] with corrupted logical plan", + path: "/api/v1/query?query=test&time=1536673670", + start: 1536673670, + end: 1536673670, + stepDuration: 0, + requestBody: func(t *testing.T) []byte { + return append(createTestLogicalPlan(t, 1536673670, 1536673670, 0), []byte("random data")...) + }, + expectedCode: http.StatusInternalServerError, + expectedBody: `{"status":"error","errorType":"server_error","error":"invalid logical plan: invalid character 'r' after top-level value"}`, + }, + { + name: "[Instant Query] with empty body and non-empty query string", + path: "/api/v1/query?query=test&time=1536673670", + start: 1536673670, + end: 1536673670, + stepDuration: 0, + requestBody: func(t *testing.T) []byte { + return []byte{} + }, + expectedCode: http.StatusOK, + expectedBody: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"test","foo":"bar"},"value":[1536673670,"1"]}]}}`, + }, + { + name: "[Instant Query] with empty body and empty query string", + path: "/api/v1/query?query=&time=1536673670", + start: 1536673670, + end: 1536673670, + stepDuration: 0, + requestBody: func(t *testing.T) []byte { + return []byte{} + }, + expectedCode: http.StatusBadRequest, + expectedBody: "{\"status\":\"error\",\"errorType\":\"bad_data\",\"error\":\"invalid parameter \\\"query\\\"; unknown position: parse error: no expression found in input\"}", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + c := NewQueryAPI(engine, &distributed_execution.QueryResultCache{}, mockQueryable, querier.StatsRenderer, log.NewNopLogger(), []v1.Codec{v1.JSONCodec{}}, regexp.MustCompile(".*"), false) + router := mux.NewRouter() + router.Path("/api/v1/query").Methods("POST").Handler(c.Wrap(c.InstantQueryHandler)) + router.Path("/api/v1/query_range").Methods("POST").Handler(c.Wrap(c.RangeQueryHandler)) + + req := createTestRequest(tt.path, tt.requestBody(t)) + rec := httptest.NewRecorder() + router.ServeHTTP(rec, req) + + require.Equal(t, tt.expectedCode, rec.Code) + body, err := io.ReadAll(rec.Body) + require.NoError(t, err) + require.Equal(t, tt.expectedBody, string(body)) + }) + } +} + +func createTestRequest(path string, planBytes []byte) *http.Request { + form := url.Values{} + form.Set("plan", string(planBytes)) + req := httptest.NewRequest(http.MethodPost, path, io.NopCloser(strings.NewReader(form.Encode()))) + + req.Header.Set("Content-Type", "application/x-www-form-urlencoded") + ctx := context.Background() + _, ctx = stats.ContextWithEmptyStats(ctx) + return req.WithContext(user.InjectOrgID(ctx, "user1")) +} + +func createTestLogicalPlan(t *testing.T, start, end int64, stepDuration int64) []byte { + startTime, endTime := convertMsToTime(start), convertMsToTime(end) + step := convertMsToDuration(stepDuration) + + qOpts := query.Options{ + Start: startTime, + End: startTime, + Step: 0, + StepsBatch: 10, + LookbackDelta: 0, + EnablePerStepStats: false, + } + + if step != 0 { + qOpts.End = endTime + qOpts.Step = step + } + + // using a different metric name here so that we can check with debugger which query (from query string vs http request body) + // is being executed by the queriers + expr, err := parser.NewParser("up", parser.WithFunctions(parser.Functions)).ParseExpr() + require.NoError(t, err) + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: false, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + byteval, err := logicalplan.Marshal(logicalPlan.Root()) + require.NoError(t, err) + + return byteval +} diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 09634c05b08..f50fcf26e17 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -14,7 +14,6 @@ import ( "github.com/go-kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/promql" prom_storage "github.com/prometheus/prometheus/storage" "github.com/weaveworks/common/server" "github.com/weaveworks/common/signals" @@ -35,6 +34,7 @@ import ( "github.com/cortexproject/cortex/pkg/cortex/storage" "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/distributor" + "github.com/cortexproject/cortex/pkg/engine" "github.com/cortexproject/cortex/pkg/flusher" "github.com/cortexproject/cortex/pkg/frontend" frontendv1 "github.com/cortexproject/cortex/pkg/frontend/v1" @@ -322,7 +322,7 @@ type Cortex struct { QuerierQueryable prom_storage.SampleAndChunkQueryable ExemplarQueryable prom_storage.ExemplarQueryable MetadataQuerier querier.MetadataQuerier - QuerierEngine promql.QueryEngine + QuerierEngine engine.QueryEngine QueryFrontendTripperware tripperware.Tripperware ResourceMonitor *resource.Monitor diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index c8f7e1de6ed..7344b339dba 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -4,11 +4,16 @@ import ( "context" "flag" "fmt" - + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" + "github.com/cortexproject/cortex/pkg/ring/client" "log/slog" + "net" "net/http" "runtime" "runtime/debug" + "strconv" + "strings" + "time" "github.com/go-kit/log/level" "github.com/opentracing-contrib/go-stdlib/nethttp" @@ -361,6 +366,37 @@ func (t *Cortex) initTenantFederation() (serv services.Service, err error) { // │ │ // └──────────────────┘ func (t *Cortex) initQuerier() (serv services.Service, err error) { + + t.Cfg.Worker.MaxConcurrentRequests = t.Cfg.Querier.MaxConcurrent + t.Cfg.Worker.TargetHeaders = t.Cfg.API.HTTPRequestHeadersToLog + + ipAddr, err := ring.GetInstanceAddr(t.Cfg.Alertmanager.ShardingRing.InstanceAddr, t.Cfg.Alertmanager.ShardingRing.InstanceInterfaceNames, util_log.Logger) + if err != nil { + return nil, err + } + serverAddress := net.JoinHostPort(ipAddr, strconv.Itoa(t.Cfg.Server.GRPCListenPort)) + + // Create new map for caching partial results during distributed execution + var queryResultCache *distributed_execution.QueryResultCache + var queryServer *distributed_execution.QuerierServer + + if t.Cfg.Querier.DistributedExecEnabled { + // set up querier server service and register it + queryResultCache = distributed_execution.NewQueryResultCache() + queryServer = distributed_execution.NewQuerierServer(queryResultCache) + + go func() { + // TODO: make expire time a config var + ticker := time.NewTicker(5 * time.Minute) + defer ticker.Stop() + + for range ticker.C { + queryResultCache.CleanExpired() + } + }() + t.API.RegisterQuerierServer(queryServer) + } + // Create a internal HTTP handler that is configured with the Prometheus API routes and points // to a Prometheus API struct instantiated with the Cortex Queryable. internalQuerierRouter := api.NewQuerierHandler( @@ -368,9 +404,11 @@ func (t *Cortex) initQuerier() (serv services.Service, err error) { t.QuerierQueryable, t.ExemplarQueryable, t.QuerierEngine, + queryResultCache, t.MetadataQuerier, prometheus.DefaultRegisterer, util_log.Logger, + t.Cfg.Querier.DistributedExecEnabled, ) // If the querier is running standalone without the query-frontend or query-scheduler, we must register it's internal @@ -411,9 +449,25 @@ func (t *Cortex) initQuerier() (serv services.Service, err error) { return nil, nil } - t.Cfg.Worker.MaxConcurrentRequests = t.Cfg.Querier.MaxConcurrent - t.Cfg.Worker.TargetHeaders = t.Cfg.API.HTTPRequestHeadersToLog - return querier_worker.NewQuerierWorker(t.Cfg.Worker, httpgrpc_server.NewServer(internalQuerierRouter), util_log.Logger, prometheus.DefaultRegisterer) + if t.Cfg.Querier.DistributedExecEnabled { + querierPool := distributed_execution.NewQuerierPool(t.Cfg.QueryScheduler.GRPCClientConfig, prometheus.DefaultRegisterer, util_log.Logger) + internalQuerierRouter = injectPool(internalQuerierRouter, querierPool) + //go watchQuerierRingAndUpdatePool(context.Background(), t.Ring, querierPool) + } + + return querier_worker.NewQuerierWorker(t.Cfg.Worker, httpgrpc_server.NewServer(internalQuerierRouter), util_log.Logger, prometheus.DefaultRegisterer, serverAddress, t.Cfg.Querier.DistributedExecEnabled, queryResultCache) +} + +func injectPool(next http.Handler, pool *client.Pool) http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.ProtoMajor == 2 && strings.Contains(r.Header.Get("Content-Type"), "application/grpc") { + ctx := distributed_execution.ContextWithPool(r.Context(), pool) + next.ServeHTTP(w, r.WithContext(ctx)) + return + } + ctx := distributed_execution.ContextWithPool(r.Context(), pool) + next.ServeHTTP(w, r.WithContext(ctx)) + }) } func (t *Cortex) initStoreQueryables() (services.Service, error) { @@ -539,7 +593,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, t.Cfg.Querier.DefaultEvaluationInterval, - t.Cfg.Frontend.DistributedExecEnabled, + t.Cfg.Querier.DistributedExecEnabled, ) if err != nil { return nil, err @@ -552,7 +606,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Querier.DefaultEvaluationInterval, - t.Cfg.Frontend.DistributedExecEnabled) + t.Cfg.Querier.DistributedExecEnabled) if err != nil { return nil, err } @@ -811,7 +865,7 @@ func (t *Cortex) initQueryScheduler() (services.Service, error) { tenant.WithDefaultResolver(tenantfederation.NewRegexValidator()) } - s, err := scheduler.NewScheduler(t.Cfg.QueryScheduler, t.Overrides, util_log.Logger, prometheus.DefaultRegisterer) + s, err := scheduler.NewScheduler(t.Cfg.QueryScheduler, t.Overrides, util_log.Logger, prometheus.DefaultRegisterer, t.Cfg.Querier.DistributedExecEnabled) if err != nil { return nil, errors.Wrap(err, "query-scheduler init") } diff --git a/pkg/engine/distributed_execution/id.go b/pkg/engine/distributed_execution/id.go new file mode 100644 index 00000000000..ed09c3faf38 --- /dev/null +++ b/pkg/engine/distributed_execution/id.go @@ -0,0 +1,37 @@ +package distributed_execution + +import ( + "context" +) + +type fragmentMetadataKey struct{} + +type fragmentMetadata struct { + queryID uint64 + fragmentID uint64 + childIDToAddr map[uint64]string + isRoot bool +} + +func InjectFragmentMetaData(ctx context.Context, fragmentID uint64, queryID uint64, isRoot bool, childIDs []uint64, childAddr []string) context.Context { + + childIDToAddr := make(map[uint64]string, len(childIDs)) + for i, childID := range childIDs { + childIDToAddr[childID] = childAddr[i] + } + + return context.WithValue(ctx, fragmentMetadataKey{}, fragmentMetadata{ + queryID: queryID, + fragmentID: fragmentID, + childIDToAddr: childIDToAddr, + isRoot: isRoot, + }) +} + +func ExtractFragmentMetaData(ctx context.Context) (isRoot bool, queryID uint64, fragmentID uint64, childAddrs map[uint64]string, ok bool) { + metadata, ok := ctx.Value(fragmentMetadataKey{}).(fragmentMetadata) + if !ok { + return false, 0, 0, nil, false + } + return metadata.isRoot, metadata.queryID, metadata.fragmentID, metadata.childIDToAddr, true +} diff --git a/pkg/engine/distributed_execution/id_test.go b/pkg/engine/distributed_execution/id_test.go new file mode 100644 index 00000000000..a93f778a536 --- /dev/null +++ b/pkg/engine/distributed_execution/id_test.go @@ -0,0 +1,92 @@ +package distributed_execution + +import ( + "context" + "reflect" + "testing" +) + +func TestFragmentMetadata(t *testing.T) { + tests := []struct { + name string + queryID uint64 + fragID uint64 + isRoot bool + childIDs []uint64 + childAddr []string + }{ + { + name: "basic test", + queryID: 123, + fragID: 456, + isRoot: true, + childIDs: []uint64{1, 2, 3}, + childAddr: []string{"addr1", "addr2", "addr3"}, + }, + { + name: "empty children", + queryID: 789, + fragID: 101, + isRoot: false, + childIDs: []uint64{}, + childAddr: []string{}, + }, + { + name: "single child", + queryID: 999, + fragID: 888, + isRoot: true, + childIDs: []uint64{42}, + childAddr: []string{"[IP_ADDRESS]:8080"}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // injection + ctx := context.Background() + newCtx := InjectFragmentMetaData(ctx, tt.fragID, tt.queryID, tt.isRoot, tt.childIDs, tt.childAddr) + + // extraction + isRoot, queryID, fragmentID, childAddrs, ok := ExtractFragmentMetaData(newCtx) + + // verify results + if !ok { + t.Error("ExtractFragmentMetaData failed, ok = false") + } + + if isRoot != tt.isRoot { + t.Errorf("isRoot = %v, want %v", isRoot, tt.isRoot) + } + + if queryID != tt.queryID { + t.Errorf("queryID = %v, want %v", queryID, tt.queryID) + } + + if fragmentID != tt.fragID { + t.Errorf("fragmentID = %v, want %v", fragmentID, tt.fragID) + } + + // create expected childIDToAddr map + expectedChildAddrs := make(map[uint64]string) + for i, childID := range tt.childIDs { + expectedChildAddrs[childID] = tt.childAddr[i] + } + + if !reflect.DeepEqual(childAddrs, expectedChildAddrs) { + t.Errorf("childAddrs = %v, want %v", childAddrs, expectedChildAddrs) + } + }) + } +} + +func TestExtractFragmentMetaDataWithEmptyContext(t *testing.T) { + ctx := context.Background() + isRoot, queryID, fragmentID, childAddrs, ok := ExtractFragmentMetaData(ctx) + if ok { + t.Error("ExtractFragmentMetaData should return ok=false for empty context") + } + if isRoot || queryID != 0 || fragmentID != 0 || childAddrs != nil { + t.Error("ExtractFragmentMetaData should return zero values for empty context") + } +} diff --git a/pkg/engine/distributed_execution/logicalplan.go b/pkg/engine/distributed_execution/logicalplan.go new file mode 100644 index 00000000000..77a73026d9c --- /dev/null +++ b/pkg/engine/distributed_execution/logicalplan.go @@ -0,0 +1,231 @@ +package distributed_execution + +import ( + "bytes" + "encoding/json" + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/promql-engine/logicalplan" + "math" +) + +type jsonNode struct { + Type logicalplan.NodeType `json:"type"` + Data json.RawMessage `json:"data"` + Children []json.RawMessage `json:"children,omitempty"` +} + +const ( + nanVal = `"NaN"` + infVal = `"+Inf"` + negInfVal = `"-Inf"` +) + +func Marshal(node Node) ([]byte, error) { + clone := node.Clone() + return marshalNode(clone) +} + +func marshalNode(node Node) ([]byte, error) { + children := make([]json.RawMessage, 0, len(node.Children())) + for _, c := range node.Children() { + childData, err := marshalNode(*c) + if err != nil { + return nil, err + } + children = append(children, childData) + } + var data json.RawMessage = nil + if n, ok := node.(*logicalplan.NumberLiteral); ok { + if math.IsInf(n.Val, 1) { + data = json.RawMessage(infVal) + } + if math.IsInf(n.Val, -1) { + data = json.RawMessage(negInfVal) + } + if math.IsNaN(n.Val) { + data = json.RawMessage(nanVal) + } + } + if data == nil { + var err error + data, err = json.Marshal(node) + if err != nil { + return nil, err + } + } + return json.Marshal(jsonNode{ + Type: node.Type(), + Data: data, + Children: children, + }) +} + +func Unmarshal(data []byte) (logicalplan.Node, error) { + return unmarshalNode(data) +} + +func unmarshalNode(data []byte) (logicalplan.Node, error) { + t := jsonNode{} + if err := json.Unmarshal(data, &t); err != nil { + return nil, err + } + + switch t.Type { + case logicalplan.VectorSelectorNode: + v := &logicalplan.VectorSelector{} + if err := json.Unmarshal(t.Data, v); err != nil { + return nil, err + } + var err error + for i, m := range v.LabelMatchers { + v.LabelMatchers[i], err = labels.NewMatcher(m.Type, m.Name, m.Value) + if err != nil { + return nil, err + } + } + return v, nil + case logicalplan.MatrixSelectorNode: + m := &logicalplan.MatrixSelector{} + if err := json.Unmarshal(t.Data, m); err != nil { + return nil, err + } + vs, err := unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + m.VectorSelector = vs.(*logicalplan.VectorSelector) + return m, nil + case logicalplan.AggregationNode: + a := &logicalplan.Aggregation{} + if err := json.Unmarshal(t.Data, a); err != nil { + return nil, err + } + var err error + a.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + if len(t.Children) > 1 { + a.Param, err = unmarshalNode(t.Children[1]) + if err != nil { + return nil, err + } + } + return a, nil + case logicalplan.BinaryNode: + b := &logicalplan.Binary{} + if err := json.Unmarshal(t.Data, b); err != nil { + return nil, err + } + var err error + b.LHS, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + b.RHS, err = unmarshalNode(t.Children[1]) + if err != nil { + return nil, err + } + return b, nil + case logicalplan.FunctionNode: + f := &logicalplan.FunctionCall{} + if err := json.Unmarshal(t.Data, f); err != nil { + return nil, err + } + for _, c := range t.Children { + child, err := unmarshalNode(c) + if err != nil { + return nil, err + } + f.Args = append(f.Args, child) + } + return f, nil + case logicalplan.NumberLiteralNode: + n := &logicalplan.NumberLiteral{} + if bytes.Equal(t.Data, []byte(infVal)) { + n.Val = math.Inf(1) + } else if bytes.Equal(t.Data, []byte(negInfVal)) { + n.Val = math.Inf(-1) + } else if bytes.Equal(t.Data, []byte(nanVal)) { + n.Val = math.NaN() + } else { + if err := json.Unmarshal(t.Data, n); err != nil { + return nil, err + } + } + return n, nil + case logicalplan.StringLiteralNode: + s := &logicalplan.StringLiteral{} + if err := json.Unmarshal(t.Data, s); err != nil { + return nil, err + } + return s, nil + case logicalplan.SubqueryNode: + s := &logicalplan.Subquery{} + if err := json.Unmarshal(t.Data, s); err != nil { + return nil, err + } + var err error + s.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return s, nil + case logicalplan.CheckDuplicateNode: + c := &logicalplan.CheckDuplicateLabels{} + if err := json.Unmarshal(t.Data, c); err != nil { + return nil, err + } + var err error + c.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return c, nil + case logicalplan.StepInvariantNode: + s := &logicalplan.StepInvariantExpr{} + if err := json.Unmarshal(t.Data, s); err != nil { + return nil, err + } + var err error + s.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return s, nil + case logicalplan.ParensNode: + p := &logicalplan.Parens{} + if err := json.Unmarshal(t.Data, p); err != nil { + return nil, err + } + var err error + p.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return p, nil + case logicalplan.UnaryNode: + u := &logicalplan.Unary{} + if err := json.Unmarshal(t.Data, u); err != nil { + return nil, err + } + var err error + u.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return u, nil + case RemoteNode: + r := &Remote{} + if err := json.Unmarshal(t.Data, r); err != nil { + return nil, err + } + var err error + r.Expr, err = unmarshalNode(t.Children[0]) + if err != nil { + return nil, err + } + return r, nil + } + return nil, nil +} diff --git a/pkg/engine/distributed_execution/logicalplan_test.go b/pkg/engine/distributed_execution/logicalplan_test.go new file mode 100644 index 00000000000..e2d8d5962ab --- /dev/null +++ b/pkg/engine/distributed_execution/logicalplan_test.go @@ -0,0 +1 @@ +package distributed_execution diff --git a/pkg/engine/distributed_execution/querier_service_client.go b/pkg/engine/distributed_execution/querier_service_client.go new file mode 100644 index 00000000000..b186a5fedf0 --- /dev/null +++ b/pkg/engine/distributed_execution/querier_service_client.go @@ -0,0 +1,119 @@ +package distributed_execution + +import ( + "github.com/cortexproject/cortex/pkg/engine/distributed_execution/querierpb" + "github.com/cortexproject/cortex/pkg/ring/client" + "github.com/cortexproject/cortex/pkg/util/grpcclient" + cortexmiddleware "github.com/cortexproject/cortex/pkg/util/middleware" + "github.com/go-kit/log" + otgrpc "github.com/opentracing-contrib/go-grpc" + "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/model/histogram" + "github.com/weaveworks/common/middleware" + "google.golang.org/grpc" + "google.golang.org/grpc/health/grpc_health_v1" + "time" +) + +type querierClient struct { + querierpb.QuerierClient + grpc_health_v1.HealthClient + conn *grpc.ClientConn +} + +func (qc *querierClient) Close() error { + return qc.conn.Close() +} + +func NewQuerierPool(cfg grpcclient.Config, reg prometheus.Registerer, log log.Logger) *client.Pool { + requestDuration := promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "cortex_querier_query_request_duration_seconds", + Help: "Time spent doing requests to querier.", + Buckets: prometheus.ExponentialBuckets(0.001, 4, 6), + }, []string{"operation", "status_code"}) + + clientsGauge := promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_querier_query_clients", + Help: "The current number of clients connected to querier.", + }) + + poolConfig := client.PoolConfig{ + CheckInterval: time.Minute, + HealthCheckEnabled: true, + HealthCheckTimeout: 10 * time.Second, + } + + q := &querierPool{ + grpcConfig: cfg, + requestDuration: requestDuration, + } + + return client.NewPool("querier", poolConfig, nil, q.createQuerierClient, clientsGauge, log) +} + +type querierPool struct { + grpcConfig grpcclient.Config + requestDuration *prometheus.HistogramVec +} + +func (q *querierPool) createQuerierClient(addr string) (client.PoolClient, error) { + opts, err := q.grpcConfig.DialOption([]grpc.UnaryClientInterceptor{ + otgrpc.OpenTracingClientInterceptor(opentracing.GlobalTracer()), + middleware.ClientUserHeaderInterceptor, + cortexmiddleware.PrometheusGRPCUnaryInstrumentation(q.requestDuration), + }, nil) + + if err != nil { + return nil, err + } + + conn, err := grpc.NewClient(addr, opts...) + if err != nil { + return nil, err + } + + return &querierClient{ + QuerierClient: querierpb.NewQuerierClient(conn), + HealthClient: grpc_health_v1.NewHealthClient(conn), + conn: conn, + }, nil +} + +func FloatHistogramProtoToFloatHistograms(hps []querierpb.Histogram) []*histogram.FloatHistogram { + floatHistograms := make([]*histogram.FloatHistogram, len(hps)) + for _, hp := range hps { + newHist := FloatHistogramProtoToFloatHistogram(hp) + floatHistograms = append(floatHistograms, newHist) + } + return floatHistograms +} + +func FloatHistogramProtoToFloatHistogram(hp querierpb.Histogram) *histogram.FloatHistogram { + _, IsFloatHist := hp.GetCount().(*querierpb.Histogram_CountFloat) + if !IsFloatHist { + panic("FloatHistogramProtoToFloatHistogram called with an integer histogram") + } + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(hp.ResetHint), + Schema: hp.Schema, + ZeroThreshold: hp.ZeroThreshold, + ZeroCount: hp.GetZeroCountFloat(), + Count: hp.GetCountFloat(), + Sum: hp.Sum, + PositiveSpans: spansProtoToSpans(hp.GetPositiveSpans()), + PositiveBuckets: hp.GetPositiveCounts(), + NegativeSpans: spansProtoToSpans(hp.GetNegativeSpans()), + NegativeBuckets: hp.GetNegativeCounts(), + } +} + +func spansProtoToSpans(s []querierpb.BucketSpan) []histogram.Span { + spans := make([]histogram.Span, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} diff --git a/pkg/engine/distributed_execution/querier_service_server.go b/pkg/engine/distributed_execution/querier_service_server.go new file mode 100644 index 00000000000..892b41c2eef --- /dev/null +++ b/pkg/engine/distributed_execution/querier_service_server.go @@ -0,0 +1,295 @@ +package distributed_execution + +import ( + "fmt" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + v1 "github.com/prometheus/prometheus/web/api/v1" + "time" + + "github.com/prometheus/prometheus/model/histogram" + + "github.com/cortexproject/cortex/pkg/engine/distributed_execution/querierpb" +) + +const ( + BATCHSIZE = 1000 + WritingTimeout = 100 * time.Millisecond + MaxRetries = 3 + RetryDelay = 100 * time.Millisecond +) + +type QuerierServer struct { + queryResultCache *QueryResultCache +} + +func NewQuerierServer(cache *QueryResultCache) *QuerierServer { + return &QuerierServer{ + queryResultCache: cache, + } +} + +func (s *QuerierServer) Series(req *querierpb.SeriesRequest, srv querierpb.Querier_SeriesServer) error { + key := MakeFragmentKey(req.QueryID, req.FragmentID) + + for { + var result FragmentResult + var ok bool + for attempt := 1; attempt <= MaxRetries; attempt++ { + result, ok = s.queryResultCache.Get(key) + if ok { + break + } + if attempt == MaxRetries { + return fmt.Errorf("fragment not found after %d attempts: %v", MaxRetries, key) + } + time.Sleep(RetryDelay) + } + + switch result.Status { + case StatusDone: + v1ResultData := result.Data.(*v1.QueryData) + + switch v1ResultData.ResultType { + case parser.ValueTypeMatrix: + series := v1ResultData.Result.(promql.Matrix) + + seriesBatch := []*querierpb.OneSeries{} + for _, s := range series { + oneSeries := &querierpb.OneSeries{ + Labels: make([]*querierpb.Label, len(s.Metric)), + } + for j, l := range s.Metric { + oneSeries.Labels[j] = &querierpb.Label{ + Name: l.Name, + Value: l.Value} + } + seriesBatch = append(seriesBatch, oneSeries) + } + if err := srv.Send(&querierpb.SeriesBatch{ + OneSeries: seriesBatch}); err != nil { + return err + } + + return nil + + case parser.ValueTypeVector: + samples := v1ResultData.Result.(promql.Vector) + + seriesBatch := []*querierpb.OneSeries{} + for _, s := range samples { + oneSeries := &querierpb.OneSeries{ + Labels: make([]*querierpb.Label, len(s.Metric)), + } + for j, l := range s.Metric { + oneSeries.Labels[j] = &querierpb.Label{ + Name: l.Name, + Value: l.Value, + } + } + seriesBatch = append(seriesBatch, oneSeries) + } + if err := srv.Send(&querierpb.SeriesBatch{ + OneSeries: seriesBatch, + }); err != nil { + return err + } + return nil + } + + case StatusError: + return fmt.Errorf("fragment processing failed") + + case StatusWriting: + time.Sleep(WritingTimeout) + continue + } + } +} + +func (s *QuerierServer) Next(req *querierpb.NextRequest, srv querierpb.Querier_NextServer) error { + key := MakeFragmentKey(req.QueryID, req.FragmentID) + + batchSize := int(req.Batchsize) + if batchSize <= 0 { + batchSize = BATCHSIZE + } + + for { + result, ok := s.queryResultCache.Get(key) + + for attempt := 1; attempt <= MaxRetries; attempt++ { + result, ok = s.queryResultCache.Get(key) + if ok { + break + } + if attempt == MaxRetries { + return fmt.Errorf("fragment not found after %d attempts: %v", MaxRetries, key) + } + time.Sleep(RetryDelay) + } + + switch result.Status { + case StatusDone: + v1ResultData := result.Data.(*v1.QueryData) + + switch v1ResultData.ResultType { + case parser.ValueTypeMatrix: + matrix := v1ResultData.Result.(promql.Matrix) + + numTimeSteps := matrix.TotalSamples() + + for timeStep := 0; timeStep < numTimeSteps; timeStep += batchSize { + batch := &querierpb.StepVectorBatch{ + StepVectors: make([]*querierpb.StepVector, 0, len(matrix)), + } + for t := 0; t < batchSize; t++ { + for i, series := range matrix { + vector, err := s.createVectorForTimestep(&series, timeStep+t, uint64(i)) + if err != nil { + return err + } + batch.StepVectors = append(batch.StepVectors, vector) + } + } + if err := srv.Send(batch); err != nil { + return fmt.Errorf("error sending batch: %w", err) + } + } + return nil + + case parser.ValueTypeVector: + vector := v1ResultData.Result.(promql.Vector) + + for i := 0; i < len(vector); i += batchSize { + end := i + batchSize + if end > len(vector) { + end = len(vector) + } + + batch := &querierpb.StepVectorBatch{ + StepVectors: []*querierpb.StepVector{}, + } + + var timestamp int64 + sampleIDs := make([]uint64, 0, batchSize) + samples := make([]float64, 0, batchSize) + histogramIDs := make([]uint64, 0, batchSize) + histograms := make([]*histogram.FloatHistogram, 0, batchSize) + + for j, sample := range (vector)[i:end] { + if sample.H == nil { + sampleIDs = append(sampleIDs, uint64(j)) + samples = append(samples, sample.F) + } else { + histogramIDs = append(histogramIDs, uint64(j)) + histograms = append(histograms, sample.H) + } + } + vec := &querierpb.StepVector{ + T: timestamp, + Sample_IDs: sampleIDs, + Samples: samples, + Histogram_IDs: histogramIDs, + Histograms: FloatHistogramsToFloatHistogramProto(histograms), + } + batch.StepVectors = append(batch.StepVectors, vec) + if err := srv.Send(batch); err != nil { + return err + } + } + return nil + + default: + return fmt.Errorf("unsupported result type: %v", v1ResultData.ResultType) + } + case StatusError: + return fmt.Errorf("fragment processing failed") + case StatusWriting: + time.Sleep(WritingTimeout) + continue + } + } +} + +func (s *QuerierServer) createVectorForTimestep(series *promql.Series, timeStep int, sampleID uint64) (*querierpb.StepVector, error) { + var samples []float64 + var sampleIDs []uint64 + var histograms []*histogram.FloatHistogram + var histogramIDs []uint64 + var timestamp int64 + + if timeStep < len(series.Floats) { + point := series.Floats[timeStep] + timestamp = point.T + samples = append(samples, point.F) + sampleIDs = append(sampleIDs, sampleID) + } + + if timeStep < len(series.Histograms) { + point := series.Histograms[timeStep] + timestamp = point.T + histograms = append(histograms, point.H) + histogramIDs = append(histogramIDs, uint64(timeStep)) + } + + return &querierpb.StepVector{ + T: timestamp, + Sample_IDs: sampleIDs, + Samples: samples, + Histogram_IDs: histogramIDs, + Histograms: FloatHistogramsToFloatHistogramProto(histograms), + }, nil +} + +func FloatHistogramsToFloatHistogramProto(histograms []*histogram.FloatHistogram) []querierpb.Histogram { + if histograms == nil { + return []querierpb.Histogram{} + } + + protoHistograms := make([]querierpb.Histogram, 0, len(histograms)) + for _, h := range histograms { + if h != nil { + protoHist := FloatHistogramToFloatHistogramProto(h) + protoHistograms = append(protoHistograms, *protoHist) + } + } + return protoHistograms +} + +func FloatHistogramToFloatHistogramProto(h *histogram.FloatHistogram) *querierpb.Histogram { + if h == nil { + return nil + } + + return &querierpb.Histogram{ + ResetHint: querierpb.Histogram_ResetHint(h.CounterResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + Count: &querierpb.Histogram_CountFloat{ + CountFloat: h.Count, + }, + ZeroCount: &querierpb.Histogram_ZeroCountFloat{ + ZeroCountFloat: h.ZeroCount, + }, + Sum: h.Sum, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveCounts: h.PositiveBuckets, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeCounts: h.NegativeBuckets, + } +} + +func spansToSpansProto(spans []histogram.Span) []querierpb.BucketSpan { + if spans == nil { + return nil + } + protoSpans := make([]querierpb.BucketSpan, len(spans)) + for i, span := range spans { + protoSpans[i] = querierpb.BucketSpan{ + Offset: span.Offset, + Length: span.Length, + } + } + return protoSpans +} diff --git a/pkg/engine/distributed_execution/querier_service_server_test.go b/pkg/engine/distributed_execution/querier_service_server_test.go new file mode 100644 index 00000000000..8e30ac5dd4d --- /dev/null +++ b/pkg/engine/distributed_execution/querier_service_server_test.go @@ -0,0 +1,202 @@ +package distributed_execution + +import ( + "context" + "github.com/prometheus/prometheus/model/labels" + "testing" + + "github.com/cortexproject/cortex/pkg/engine/distributed_execution/querierpb" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + v1 "github.com/prometheus/prometheus/web/api/v1" + "github.com/stretchr/testify/assert" +) + +func TestQuerierServer_Series(t *testing.T) { + tests := []struct { + name string + setupCache func() *QueryResultCache + request *querierpb.SeriesRequest + wantErr bool + errMessage string + }{ + { + name: "matrix data type success", + setupCache: func() *QueryResultCache { + cache := NewQueryResultCache() + matrix := promql.Matrix{ + promql.Series{ + Metric: labels.FromStrings("__name__", "foo"), + Floats: []promql.FPoint{{F: 1, T: 1000}}, + }, + promql.Series{ + Metric: labels.FromStrings("__name__", "bar"), + Floats: []promql.FPoint{{F: 2, T: 2000}}, + }, + } + cache.SetComplete(MakeFragmentKey(1, 1), &v1.QueryData{ + ResultType: parser.ValueTypeMatrix, + Result: matrix, + }) + return cache + }, + request: &querierpb.SeriesRequest{ + QueryID: 1, + FragmentID: 1, + }, + wantErr: false, + }, + { + name: "vector data type success", + setupCache: func() *QueryResultCache { + cache := NewQueryResultCache() + vector := promql.Vector{promql.Sample{}} + cache.SetComplete(MakeFragmentKey(1, 1), + &v1.QueryData{ + ResultType: parser.ValueTypeVector, + Result: vector, + }) + return cache + }, + request: &querierpb.SeriesRequest{ + QueryID: 1, + FragmentID: 1, + }, + wantErr: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cache := tt.setupCache() + server := NewQuerierServer(cache) + + mockStream := &mockSeriesServer{} + err := server.Series(tt.request, mockStream) + + if tt.wantErr { + assert.Error(t, err) + if tt.errMessage != "" { + assert.Contains(t, err.Error(), tt.errMessage) + } + } else { + assert.NoError(t, err) + } + }) + } +} + +func TestQuerierServer_Next(t *testing.T) { + tests := []struct { + name string + setupCache func() *QueryResultCache + request *querierpb.NextRequest + wantErr bool + errMessage string + }{ + { + name: "matrix data type success", + setupCache: func() *QueryResultCache { + cache := NewQueryResultCache() + matrix := promql.Matrix{ + promql.Series{ + Metric: labels.FromStrings("__name__", "foo"), + Floats: []promql.FPoint{{F: 1, T: 1000}}, + }, + promql.Series{ + Metric: labels.FromStrings("__name__", "bar"), + Floats: []promql.FPoint{{F: 2, T: 2000}}, + }, + } + cache.SetComplete(MakeFragmentKey(1, 1), &v1.QueryData{ + ResultType: parser.ValueTypeMatrix, + Result: matrix, + }) + return cache + }, + request: &querierpb.NextRequest{ + QueryID: 1, + FragmentID: 1, + Batchsize: BATCHSIZE, + }, + wantErr: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cache := tt.setupCache() + server := NewQuerierServer(cache) + + mockStream := &mockNextServer{} + err := server.Next(tt.request, mockStream) + + if tt.wantErr { + assert.Error(t, err) + if tt.errMessage != "" { + assert.Contains(t, err.Error(), tt.errMessage) + } + } else { + assert.NoError(t, err) + } + }) + } +} + +func TestFloatHistogramConversion(t *testing.T) { + original := &histogram.FloatHistogram{ + Schema: 1, + ZeroThreshold: 0.001, + ZeroCount: 2.0, + Count: 10.0, + Sum: 100.0, + PositiveSpans: []histogram.Span{{Offset: 1, Length: 2}}, + PositiveBuckets: []float64{1.0, 2.0}, + NegativeSpans: []histogram.Span{{Offset: -2, Length: 1}}, + NegativeBuckets: []float64{-1.0}, + } + + proto := FloatHistogramToFloatHistogramProto(original) + + result := FloatHistogramProtoToFloatHistogram(*proto) + + assert.Equal(t, original.Schema, result.Schema) + assert.Equal(t, original.ZeroThreshold, result.ZeroThreshold) + assert.Equal(t, original.ZeroCount, result.ZeroCount) + assert.Equal(t, original.Count, result.Count) + assert.Equal(t, original.Sum, result.Sum) + assert.Equal(t, original.PositiveSpans, result.PositiveSpans) + assert.Equal(t, original.PositiveBuckets, result.PositiveBuckets) + assert.Equal(t, original.NegativeSpans, result.NegativeSpans) + assert.Equal(t, original.NegativeBuckets, result.NegativeBuckets) +} + +// mock implementations for testing +type mockSeriesServer struct { + querierpb.Querier_SeriesServer + sent []*querierpb.SeriesBatch +} + +func (m *mockSeriesServer) Send(batch *querierpb.SeriesBatch) error { + m.sent = append(m.sent, batch) + return nil +} + +func (m *mockSeriesServer) Context() context.Context { + return context.Background() +} + +type mockNextServer struct { + querierpb.Querier_NextServer + sent []*querierpb.StepVectorBatch +} + +func (m *mockNextServer) Send(batch *querierpb.StepVectorBatch) error { + m.sent = append(m.sent, batch) + return nil +} + +func (m *mockNextServer) Context() context.Context { + return context.Background() +} diff --git a/pkg/engine/distributed_execution/querierpb/querier.pb.go b/pkg/engine/distributed_execution/querierpb/querier.pb.go new file mode 100644 index 00000000000..6e026881a2d --- /dev/null +++ b/pkg/engine/distributed_execution/querierpb/querier.pb.go @@ -0,0 +1,3477 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: pkg/engine/distributed_execution/querierpb/querier.proto + +package querierpb + +import ( + context "context" + encoding_binary "encoding/binary" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type Histogram_ResetHint int32 + +const ( + Histogram_UNKNOWN Histogram_ResetHint = 0 + Histogram_YES Histogram_ResetHint = 1 + Histogram_NO Histogram_ResetHint = 2 + Histogram_GAUGE Histogram_ResetHint = 3 +) + +var Histogram_ResetHint_name = map[int32]string{ + 0: "UNKNOWN", + 1: "YES", + 2: "NO", + 3: "GAUGE", +} + +var Histogram_ResetHint_value = map[string]int32{ + "UNKNOWN": 0, + "YES": 1, + "NO": 2, + "GAUGE": 3, +} + +func (x Histogram_ResetHint) String() string { + return proto.EnumName(Histogram_ResetHint_name, int32(x)) +} + +func (Histogram_ResetHint) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{7, 0} +} + +type SeriesRequest struct { + QueryID uint64 `protobuf:"varint,1,opt,name=queryID,proto3" json:"queryID,omitempty"` + FragmentID uint64 `protobuf:"varint,2,opt,name=fragmentID,proto3" json:"fragmentID,omitempty"` + Batchsize int64 `protobuf:"varint,3,opt,name=batchsize,proto3" json:"batchsize,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } +func (m *SeriesRequest) String() string { return proto.CompactTextString(m) } +func (*SeriesRequest) ProtoMessage() {} +func (*SeriesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{0} +} +func (m *SeriesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesRequest.Merge(m, src) +} +func (m *SeriesRequest) XXX_Size() int { + return m.Size() +} +func (m *SeriesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesRequest proto.InternalMessageInfo + +func (m *SeriesRequest) GetQueryID() uint64 { + if m != nil { + return m.QueryID + } + return 0 +} + +func (m *SeriesRequest) GetFragmentID() uint64 { + if m != nil { + return m.FragmentID + } + return 0 +} + +func (m *SeriesRequest) GetBatchsize() int64 { + if m != nil { + return m.Batchsize + } + return 0 +} + +type NextRequest struct { + QueryID uint64 `protobuf:"varint,1,opt,name=queryID,proto3" json:"queryID,omitempty"` + FragmentID uint64 `protobuf:"varint,2,opt,name=fragmentID,proto3" json:"fragmentID,omitempty"` + Batchsize int64 `protobuf:"varint,3,opt,name=batchsize,proto3" json:"batchsize,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NextRequest) Reset() { *m = NextRequest{} } +func (m *NextRequest) String() string { return proto.CompactTextString(m) } +func (*NextRequest) ProtoMessage() {} +func (*NextRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{1} +} +func (m *NextRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NextRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_NextRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *NextRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_NextRequest.Merge(m, src) +} +func (m *NextRequest) XXX_Size() int { + return m.Size() +} +func (m *NextRequest) XXX_DiscardUnknown() { + xxx_messageInfo_NextRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_NextRequest proto.InternalMessageInfo + +func (m *NextRequest) GetQueryID() uint64 { + if m != nil { + return m.QueryID + } + return 0 +} + +func (m *NextRequest) GetFragmentID() uint64 { + if m != nil { + return m.FragmentID + } + return 0 +} + +func (m *NextRequest) GetBatchsize() int64 { + if m != nil { + return m.Batchsize + } + return 0 +} + +type SeriesBatch struct { + OneSeries []*OneSeries `protobuf:"bytes,1,rep,name=OneSeries,proto3" json:"OneSeries,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SeriesBatch) Reset() { *m = SeriesBatch{} } +func (m *SeriesBatch) String() string { return proto.CompactTextString(m) } +func (*SeriesBatch) ProtoMessage() {} +func (*SeriesBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{2} +} +func (m *SeriesBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SeriesBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SeriesBatch.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SeriesBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_SeriesBatch.Merge(m, src) +} +func (m *SeriesBatch) XXX_Size() int { + return m.Size() +} +func (m *SeriesBatch) XXX_DiscardUnknown() { + xxx_messageInfo_SeriesBatch.DiscardUnknown(m) +} + +var xxx_messageInfo_SeriesBatch proto.InternalMessageInfo + +func (m *SeriesBatch) GetOneSeries() []*OneSeries { + if m != nil { + return m.OneSeries + } + return nil +} + +type OneSeries struct { + Labels []*Label `protobuf:"bytes,1,rep,name=labels,proto3" json:"labels,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *OneSeries) Reset() { *m = OneSeries{} } +func (m *OneSeries) String() string { return proto.CompactTextString(m) } +func (*OneSeries) ProtoMessage() {} +func (*OneSeries) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{3} +} +func (m *OneSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OneSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_OneSeries.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *OneSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_OneSeries.Merge(m, src) +} +func (m *OneSeries) XXX_Size() int { + return m.Size() +} +func (m *OneSeries) XXX_DiscardUnknown() { + xxx_messageInfo_OneSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_OneSeries proto.InternalMessageInfo + +func (m *OneSeries) GetLabels() []*Label { + if m != nil { + return m.Labels + } + return nil +} + +type StepVectorBatch struct { + StepVectors []*StepVector `protobuf:"bytes,1,rep,name=step_vectors,json=stepVectors,proto3" json:"step_vectors,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StepVectorBatch) Reset() { *m = StepVectorBatch{} } +func (m *StepVectorBatch) String() string { return proto.CompactTextString(m) } +func (*StepVectorBatch) ProtoMessage() {} +func (*StepVectorBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{4} +} +func (m *StepVectorBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StepVectorBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StepVectorBatch.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *StepVectorBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_StepVectorBatch.Merge(m, src) +} +func (m *StepVectorBatch) XXX_Size() int { + return m.Size() +} +func (m *StepVectorBatch) XXX_DiscardUnknown() { + xxx_messageInfo_StepVectorBatch.DiscardUnknown(m) +} + +var xxx_messageInfo_StepVectorBatch proto.InternalMessageInfo + +func (m *StepVectorBatch) GetStepVectors() []*StepVector { + if m != nil { + return m.StepVectors + } + return nil +} + +type StepVector struct { + T int64 `protobuf:"varint,1,opt,name=t,proto3" json:"t,omitempty"` + Sample_IDs []uint64 `protobuf:"varint,2,rep,packed,name=sample_IDs,json=sampleIDs,proto3" json:"sample_IDs,omitempty"` + Samples []float64 `protobuf:"fixed64,3,rep,packed,name=samples,proto3" json:"samples,omitempty"` + Histogram_IDs []uint64 `protobuf:"varint,4,rep,packed,name=histogram_IDs,json=histogramIDs,proto3" json:"histogram_IDs,omitempty"` + Histograms []Histogram `protobuf:"bytes,5,rep,name=histograms,proto3" json:"histograms"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StepVector) Reset() { *m = StepVector{} } +func (m *StepVector) String() string { return proto.CompactTextString(m) } +func (*StepVector) ProtoMessage() {} +func (*StepVector) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{5} +} +func (m *StepVector) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StepVector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StepVector.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *StepVector) XXX_Merge(src proto.Message) { + xxx_messageInfo_StepVector.Merge(m, src) +} +func (m *StepVector) XXX_Size() int { + return m.Size() +} +func (m *StepVector) XXX_DiscardUnknown() { + xxx_messageInfo_StepVector.DiscardUnknown(m) +} + +var xxx_messageInfo_StepVector proto.InternalMessageInfo + +func (m *StepVector) GetT() int64 { + if m != nil { + return m.T + } + return 0 +} + +func (m *StepVector) GetSample_IDs() []uint64 { + if m != nil { + return m.Sample_IDs + } + return nil +} + +func (m *StepVector) GetSamples() []float64 { + if m != nil { + return m.Samples + } + return nil +} + +func (m *StepVector) GetHistogram_IDs() []uint64 { + if m != nil { + return m.Histogram_IDs + } + return nil +} + +func (m *StepVector) GetHistograms() []Histogram { + if m != nil { + return m.Histograms + } + return nil +} + +type Label struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Label) Reset() { *m = Label{} } +func (m *Label) String() string { return proto.CompactTextString(m) } +func (*Label) ProtoMessage() {} +func (*Label) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{6} +} +func (m *Label) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Label) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Label.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Label) XXX_Merge(src proto.Message) { + xxx_messageInfo_Label.Merge(m, src) +} +func (m *Label) XXX_Size() int { + return m.Size() +} +func (m *Label) XXX_DiscardUnknown() { + xxx_messageInfo_Label.DiscardUnknown(m) +} + +var xxx_messageInfo_Label proto.InternalMessageInfo + +func (m *Label) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *Label) GetValue() string { + if m != nil { + return m.Value + } + return "" +} + +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +type Histogram struct { + // Types that are valid to be assigned to Count: + // + // *Histogram_CountInt + // *Histogram_CountFloat + Count isHistogram_Count `protobuf_oneof:"count"` + Sum float64 `protobuf:"fixed64,3,opt,name=sum,proto3" json:"sum,omitempty"` + // The schema defines the bucket schema. Currently, valid numbers + // are -4 <= n <= 8. They are all for base-2 bucket schemas, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n logarithmic buckets. Or in other words, each + // bucket boundary is the previous boundary times 2^(2^-n). In the + // future, more bucket schemas may be added using numbers < -4 or > + // 8. + Schema int32 `protobuf:"zigzag32,4,opt,name=schema,proto3" json:"schema,omitempty"` + ZeroThreshold float64 `protobuf:"fixed64,5,opt,name=zero_threshold,json=zeroThreshold,proto3" json:"zero_threshold,omitempty"` + // Types that are valid to be assigned to ZeroCount: + // + // *Histogram_ZeroCountInt + // *Histogram_ZeroCountFloat + ZeroCount isHistogram_ZeroCount `protobuf_oneof:"zero_count"` + // Negative Buckets. + NegativeSpans []BucketSpan `protobuf:"bytes,8,rep,name=negative_spans,json=negativeSpans,proto3" json:"negative_spans"` + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + NegativeDeltas []int64 `protobuf:"zigzag64,9,rep,packed,name=negative_deltas,json=negativeDeltas,proto3" json:"negative_deltas,omitempty"` + NegativeCounts []float64 `protobuf:"fixed64,10,rep,packed,name=negative_counts,json=negativeCounts,proto3" json:"negative_counts,omitempty"` + // Positive Buckets. + PositiveSpans []BucketSpan `protobuf:"bytes,11,rep,name=positive_spans,json=positiveSpans,proto3" json:"positive_spans"` + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + PositiveDeltas []int64 `protobuf:"zigzag64,12,rep,packed,name=positive_deltas,json=positiveDeltas,proto3" json:"positive_deltas,omitempty"` + PositiveCounts []float64 `protobuf:"fixed64,13,rep,packed,name=positive_counts,json=positiveCounts,proto3" json:"positive_counts,omitempty"` + ResetHint Histogram_ResetHint `protobuf:"varint,14,opt,name=reset_hint,json=resetHint,proto3,enum=querierpb.Histogram_ResetHint" json:"reset_hint,omitempty"` + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + TimestampMs int64 `protobuf:"varint,15,opt,name=timestamp_ms,json=timestampMs,proto3" json:"timestamp_ms,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Histogram) Reset() { *m = Histogram{} } +func (m *Histogram) String() string { return proto.CompactTextString(m) } +func (*Histogram) ProtoMessage() {} +func (*Histogram) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{7} +} +func (m *Histogram) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Histogram) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Histogram.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Histogram) XXX_Merge(src proto.Message) { + xxx_messageInfo_Histogram.Merge(m, src) +} +func (m *Histogram) XXX_Size() int { + return m.Size() +} +func (m *Histogram) XXX_DiscardUnknown() { + xxx_messageInfo_Histogram.DiscardUnknown(m) +} + +var xxx_messageInfo_Histogram proto.InternalMessageInfo + +type isHistogram_Count interface { + isHistogram_Count() + MarshalTo([]byte) (int, error) + Size() int +} +type isHistogram_ZeroCount interface { + isHistogram_ZeroCount() + MarshalTo([]byte) (int, error) + Size() int +} + +type Histogram_CountInt struct { + CountInt uint64 `protobuf:"varint,1,opt,name=count_int,json=countInt,proto3,oneof" json:"count_int,omitempty"` +} +type Histogram_CountFloat struct { + CountFloat float64 `protobuf:"fixed64,2,opt,name=count_float,json=countFloat,proto3,oneof" json:"count_float,omitempty"` +} +type Histogram_ZeroCountInt struct { + ZeroCountInt uint64 `protobuf:"varint,6,opt,name=zero_count_int,json=zeroCountInt,proto3,oneof" json:"zero_count_int,omitempty"` +} +type Histogram_ZeroCountFloat struct { + ZeroCountFloat float64 `protobuf:"fixed64,7,opt,name=zero_count_float,json=zeroCountFloat,proto3,oneof" json:"zero_count_float,omitempty"` +} + +func (*Histogram_CountInt) isHistogram_Count() {} +func (*Histogram_CountFloat) isHistogram_Count() {} +func (*Histogram_ZeroCountInt) isHistogram_ZeroCount() {} +func (*Histogram_ZeroCountFloat) isHistogram_ZeroCount() {} + +func (m *Histogram) GetCount() isHistogram_Count { + if m != nil { + return m.Count + } + return nil +} +func (m *Histogram) GetZeroCount() isHistogram_ZeroCount { + if m != nil { + return m.ZeroCount + } + return nil +} + +func (m *Histogram) GetCountInt() uint64 { + if x, ok := m.GetCount().(*Histogram_CountInt); ok { + return x.CountInt + } + return 0 +} + +func (m *Histogram) GetCountFloat() float64 { + if x, ok := m.GetCount().(*Histogram_CountFloat); ok { + return x.CountFloat + } + return 0 +} + +func (m *Histogram) GetSum() float64 { + if m != nil { + return m.Sum + } + return 0 +} + +func (m *Histogram) GetSchema() int32 { + if m != nil { + return m.Schema + } + return 0 +} + +func (m *Histogram) GetZeroThreshold() float64 { + if m != nil { + return m.ZeroThreshold + } + return 0 +} + +func (m *Histogram) GetZeroCountInt() uint64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountInt); ok { + return x.ZeroCountInt + } + return 0 +} + +func (m *Histogram) GetZeroCountFloat() float64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountFloat); ok { + return x.ZeroCountFloat + } + return 0 +} + +func (m *Histogram) GetNegativeSpans() []BucketSpan { + if m != nil { + return m.NegativeSpans + } + return nil +} + +func (m *Histogram) GetNegativeDeltas() []int64 { + if m != nil { + return m.NegativeDeltas + } + return nil +} + +func (m *Histogram) GetNegativeCounts() []float64 { + if m != nil { + return m.NegativeCounts + } + return nil +} + +func (m *Histogram) GetPositiveSpans() []BucketSpan { + if m != nil { + return m.PositiveSpans + } + return nil +} + +func (m *Histogram) GetPositiveDeltas() []int64 { + if m != nil { + return m.PositiveDeltas + } + return nil +} + +func (m *Histogram) GetPositiveCounts() []float64 { + if m != nil { + return m.PositiveCounts + } + return nil +} + +func (m *Histogram) GetResetHint() Histogram_ResetHint { + if m != nil { + return m.ResetHint + } + return Histogram_UNKNOWN +} + +func (m *Histogram) GetTimestampMs() int64 { + if m != nil { + return m.TimestampMs + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Histogram) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Histogram_CountInt)(nil), + (*Histogram_CountFloat)(nil), + (*Histogram_ZeroCountInt)(nil), + (*Histogram_ZeroCountFloat)(nil), + } +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +type BucketSpan struct { + Offset int32 `protobuf:"zigzag32,1,opt,name=offset,proto3" json:"offset,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BucketSpan) Reset() { *m = BucketSpan{} } +func (m *BucketSpan) String() string { return proto.CompactTextString(m) } +func (*BucketSpan) ProtoMessage() {} +func (*BucketSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_8c8902d5f321c304, []int{8} +} +func (m *BucketSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BucketSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BucketSpan.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BucketSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_BucketSpan.Merge(m, src) +} +func (m *BucketSpan) XXX_Size() int { + return m.Size() +} +func (m *BucketSpan) XXX_DiscardUnknown() { + xxx_messageInfo_BucketSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_BucketSpan proto.InternalMessageInfo + +func (m *BucketSpan) GetOffset() int32 { + if m != nil { + return m.Offset + } + return 0 +} + +func (m *BucketSpan) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +func init() { + proto.RegisterEnum("querierpb.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) + proto.RegisterType((*SeriesRequest)(nil), "querierpb.SeriesRequest") + proto.RegisterType((*NextRequest)(nil), "querierpb.NextRequest") + proto.RegisterType((*SeriesBatch)(nil), "querierpb.SeriesBatch") + proto.RegisterType((*OneSeries)(nil), "querierpb.OneSeries") + proto.RegisterType((*StepVectorBatch)(nil), "querierpb.StepVectorBatch") + proto.RegisterType((*StepVector)(nil), "querierpb.StepVector") + proto.RegisterType((*Label)(nil), "querierpb.Label") + proto.RegisterType((*Histogram)(nil), "querierpb.Histogram") + proto.RegisterType((*BucketSpan)(nil), "querierpb.BucketSpan") +} + +func init() { + proto.RegisterFile("pkg/engine/distributed_execution/querierpb/querier.proto", fileDescriptor_8c8902d5f321c304) +} + +var fileDescriptor_8c8902d5f321c304 = []byte{ + // 828 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0xdd, 0x8e, 0xdb, 0x54, + 0x10, 0xce, 0x59, 0x3b, 0x49, 0x3d, 0xf9, 0xd9, 0xf4, 0x68, 0xa9, 0xac, 0x88, 0x86, 0xd4, 0x08, + 0xb0, 0x10, 0x4a, 0x60, 0x11, 0x52, 0xb5, 0x82, 0x4a, 0x0d, 0x29, 0xdd, 0xa8, 0x90, 0x15, 0x67, + 0x29, 0x08, 0x6e, 0x22, 0x27, 0x3b, 0xeb, 0x58, 0xf5, 0x5f, 0x7d, 0x8e, 0x57, 0xa5, 0x4f, 0x00, + 0xcf, 0xc1, 0x2b, 0xf0, 0x10, 0xbd, 0x44, 0xe2, 0x1e, 0xa1, 0x7d, 0x12, 0x74, 0x8e, 0xed, 0x63, + 0xa3, 0xed, 0x45, 0x6f, 0x7a, 0x37, 0xf3, 0xcd, 0xf7, 0x4d, 0x3e, 0x8f, 0x27, 0x63, 0xb8, 0x9f, + 0x3e, 0xf3, 0xe7, 0x18, 0xfb, 0x41, 0x8c, 0xf3, 0x8b, 0x80, 0x8b, 0x2c, 0xd8, 0xe6, 0x02, 0x2f, + 0x36, 0xf8, 0x02, 0x77, 0xb9, 0x08, 0x92, 0x78, 0xfe, 0x3c, 0xc7, 0x2c, 0xc0, 0x2c, 0xdd, 0x56, + 0xd1, 0x2c, 0xcd, 0x12, 0x91, 0x50, 0x4b, 0x17, 0xc6, 0x47, 0x7e, 0xe2, 0x27, 0x0a, 0x9d, 0xcb, + 0xa8, 0x20, 0x38, 0x3e, 0x0c, 0xce, 0x25, 0x83, 0x33, 0x7c, 0x9e, 0x23, 0x17, 0xd4, 0x86, 0xae, + 0xd4, 0xfc, 0xba, 0x5a, 0xda, 0x64, 0x4a, 0x5c, 0x93, 0x55, 0x29, 0x9d, 0x00, 0x5c, 0x66, 0x9e, + 0x1f, 0x61, 0x2c, 0x56, 0x4b, 0xfb, 0x40, 0x15, 0x1b, 0x08, 0x7d, 0x17, 0xac, 0xad, 0x27, 0x76, + 0x7b, 0x1e, 0xbc, 0x44, 0xdb, 0x98, 0x12, 0xd7, 0x60, 0x35, 0xe0, 0x20, 0xf4, 0xd6, 0xf8, 0x42, + 0xbc, 0xed, 0x9f, 0x79, 0x08, 0xbd, 0xe2, 0x79, 0x16, 0x12, 0xa2, 0xc7, 0x60, 0x9d, 0xc5, 0x58, + 0x20, 0x36, 0x99, 0x1a, 0x6e, 0xef, 0xf8, 0x68, 0xa6, 0x67, 0x32, 0xd3, 0x35, 0x56, 0xd3, 0x9c, + 0x2f, 0x1a, 0x1a, 0xea, 0x42, 0x27, 0xf4, 0xb6, 0x18, 0x56, 0xea, 0x51, 0x43, 0xfd, 0xad, 0x2c, + 0xb0, 0xb2, 0xee, 0x3c, 0x81, 0xc3, 0x73, 0x81, 0xe9, 0x8f, 0xb8, 0x13, 0x49, 0x56, 0xfc, 0xfa, + 0x7d, 0xe8, 0x73, 0x81, 0xe9, 0xe6, 0x4a, 0x61, 0x55, 0x8b, 0x77, 0x1a, 0x2d, 0x6a, 0x05, 0xeb, + 0x71, 0x1d, 0x73, 0xe7, 0x4f, 0x02, 0x50, 0xd7, 0x68, 0x1f, 0x88, 0x50, 0x73, 0x32, 0x18, 0x11, + 0xf4, 0x2e, 0x00, 0xf7, 0xa2, 0x34, 0xc4, 0xcd, 0x6a, 0xc9, 0xed, 0x83, 0xa9, 0xe1, 0x9a, 0xcc, + 0x2a, 0x90, 0xd5, 0x92, 0xcb, 0xd1, 0x16, 0x09, 0xb7, 0x8d, 0xa9, 0xe1, 0x12, 0x56, 0xa5, 0xf4, + 0x7d, 0x18, 0xec, 0x03, 0x2e, 0x12, 0x3f, 0xf3, 0x22, 0xa5, 0x35, 0x95, 0xb6, 0xaf, 0x41, 0x29, + 0x3f, 0x01, 0xd0, 0x39, 0xb7, 0xdb, 0x37, 0x66, 0x76, 0x5a, 0x15, 0x17, 0xe6, 0xab, 0x7f, 0xde, + 0x6b, 0xb1, 0x06, 0xdb, 0xf9, 0x0c, 0xda, 0x6a, 0x28, 0x94, 0x82, 0x19, 0x7b, 0x11, 0x2a, 0xcf, + 0x16, 0x53, 0x31, 0x3d, 0x82, 0xf6, 0x95, 0x17, 0xe6, 0xa8, 0xde, 0xa9, 0xc5, 0x8a, 0xc4, 0xf9, + 0xbb, 0x0d, 0x96, 0x6e, 0x49, 0xef, 0x82, 0xb5, 0x4b, 0xf2, 0x58, 0x6c, 0x82, 0xb8, 0x78, 0x60, + 0xf3, 0xb4, 0xc5, 0x6e, 0x29, 0x68, 0x15, 0x0b, 0x7a, 0x0f, 0x7a, 0x45, 0xf9, 0x32, 0x4c, 0x3c, + 0xa1, 0x1a, 0x91, 0xd3, 0x16, 0x03, 0x05, 0x7e, 0x23, 0x31, 0x3a, 0x02, 0x83, 0xe7, 0x91, 0x5a, + 0x0c, 0xc2, 0x64, 0x48, 0xef, 0x40, 0x87, 0xef, 0xf6, 0x18, 0x79, 0xb6, 0x39, 0x25, 0xee, 0x6d, + 0x56, 0x66, 0xf4, 0x03, 0x18, 0xbe, 0xc4, 0x2c, 0xd9, 0x88, 0x7d, 0x86, 0x7c, 0x9f, 0x84, 0x17, + 0x76, 0x5b, 0x89, 0x06, 0x12, 0xfd, 0xa1, 0x02, 0xe9, 0x87, 0x25, 0xad, 0xf6, 0xd5, 0x51, 0xbe, + 0x08, 0xeb, 0x4b, 0xfc, 0xeb, 0xca, 0xdb, 0xc7, 0x30, 0x6a, 0xf0, 0x0a, 0x83, 0x5d, 0x65, 0x90, + 0xb0, 0xa1, 0x66, 0x16, 0x26, 0x17, 0x30, 0x8c, 0xd1, 0xf7, 0x44, 0x70, 0x85, 0x1b, 0x9e, 0x7a, + 0x31, 0xb7, 0x6f, 0xdd, 0x58, 0x8d, 0x45, 0xbe, 0x7b, 0x86, 0xe2, 0x3c, 0xf5, 0xe2, 0x72, 0xd0, + 0x83, 0x4a, 0x22, 0x31, 0x4e, 0x3f, 0x82, 0x43, 0xdd, 0xe3, 0x02, 0x43, 0xe1, 0x71, 0xdb, 0x9a, + 0x1a, 0x2e, 0x65, 0xba, 0xf5, 0x52, 0xa1, 0xff, 0x23, 0x2a, 0x73, 0xdc, 0x06, 0xb5, 0x17, 0x9a, + 0xa8, 0x9c, 0x71, 0xe9, 0x2a, 0x4d, 0x78, 0xd0, 0x70, 0xd5, 0x7b, 0x03, 0x57, 0x95, 0x44, 0xbb, + 0xd2, 0x3d, 0x4a, 0x57, 0xfd, 0xc2, 0x55, 0x05, 0xd7, 0xae, 0x34, 0xb1, 0x74, 0x35, 0x28, 0x5c, + 0x55, 0x70, 0xe9, 0xea, 0x2b, 0x80, 0x0c, 0x39, 0x8a, 0xcd, 0x5e, 0xce, 0x7e, 0x38, 0x25, 0xee, + 0xf0, 0x78, 0xf2, 0xba, 0x7d, 0x9c, 0x31, 0x49, 0x3b, 0x0d, 0x62, 0xc1, 0xac, 0xac, 0x0a, 0xe9, + 0x3d, 0xe8, 0x8b, 0x20, 0x42, 0x2e, 0xbc, 0x28, 0xdd, 0x44, 0xdc, 0x3e, 0x54, 0xff, 0xa2, 0x9e, + 0xc6, 0xbe, 0xe3, 0xce, 0x09, 0x58, 0x5a, 0x4a, 0x7b, 0xd0, 0x7d, 0xba, 0x7e, 0xb2, 0x3e, 0xfb, + 0x69, 0x3d, 0x6a, 0xd1, 0x2e, 0x18, 0x3f, 0x3f, 0x3a, 0x1f, 0x11, 0xda, 0x81, 0x83, 0xf5, 0xd9, + 0xe8, 0x80, 0x5a, 0xd0, 0x7e, 0xfc, 0xf0, 0xe9, 0xe3, 0x47, 0x23, 0x63, 0x6c, 0xfe, 0xf6, 0xc7, + 0x84, 0x2c, 0xba, 0xd0, 0x56, 0xee, 0x17, 0x7d, 0x80, 0xfa, 0xf5, 0x3b, 0x5f, 0x02, 0xd4, 0x93, + 0x92, 0x1b, 0x98, 0x5c, 0x5e, 0x72, 0x2c, 0x56, 0xfa, 0x36, 0x2b, 0x33, 0x89, 0x87, 0x18, 0xfb, + 0x62, 0xaf, 0x36, 0x79, 0xc0, 0xca, 0xec, 0xf8, 0x77, 0x02, 0xdd, 0xef, 0x8b, 0x07, 0xa4, 0x0f, + 0xa0, 0x53, 0x9e, 0x22, 0xbb, 0x79, 0x37, 0x9a, 0x37, 0x7b, 0x7c, 0xe7, 0x46, 0x45, 0xdd, 0x1f, + 0xa7, 0xf5, 0x29, 0xa1, 0x0f, 0xc0, 0x94, 0x77, 0x97, 0x36, 0x39, 0x8d, 0x43, 0x3c, 0x1e, 0xbf, + 0xf6, 0x1a, 0x69, 0xfd, 0xe2, 0xe4, 0xd5, 0xf5, 0x84, 0xfc, 0x75, 0x3d, 0x21, 0xff, 0x5e, 0x4f, + 0xc8, 0x2f, 0x9f, 0xcc, 0xe6, 0x6f, 0xfe, 0x2d, 0xda, 0x76, 0xd4, 0x37, 0xe6, 0xf3, 0xff, 0x02, + 0x00, 0x00, 0xff, 0xff, 0xed, 0x20, 0x10, 0x11, 0xc0, 0x06, 0x00, 0x00, +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// QuerierClient is the client API for Querier service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type QuerierClient interface { + Series(ctx context.Context, in *SeriesRequest, opts ...grpc.CallOption) (Querier_SeriesClient, error) + Next(ctx context.Context, in *NextRequest, opts ...grpc.CallOption) (Querier_NextClient, error) +} + +type querierClient struct { + cc *grpc.ClientConn +} + +func NewQuerierClient(cc *grpc.ClientConn) QuerierClient { + return &querierClient{cc} +} + +func (c *querierClient) Series(ctx context.Context, in *SeriesRequest, opts ...grpc.CallOption) (Querier_SeriesClient, error) { + stream, err := c.cc.NewStream(ctx, &_Querier_serviceDesc.Streams[0], "/querierpb.Querier/Series", opts...) + if err != nil { + return nil, err + } + x := &querierSeriesClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Querier_SeriesClient interface { + Recv() (*SeriesBatch, error) + grpc.ClientStream +} + +type querierSeriesClient struct { + grpc.ClientStream +} + +func (x *querierSeriesClient) Recv() (*SeriesBatch, error) { + m := new(SeriesBatch) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +func (c *querierClient) Next(ctx context.Context, in *NextRequest, opts ...grpc.CallOption) (Querier_NextClient, error) { + stream, err := c.cc.NewStream(ctx, &_Querier_serviceDesc.Streams[1], "/querierpb.Querier/Next", opts...) + if err != nil { + return nil, err + } + x := &querierNextClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Querier_NextClient interface { + Recv() (*StepVectorBatch, error) + grpc.ClientStream +} + +type querierNextClient struct { + grpc.ClientStream +} + +func (x *querierNextClient) Recv() (*StepVectorBatch, error) { + m := new(StepVectorBatch) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// QuerierServer is the server API for Querier service. +type QuerierServer interface { + Series(*SeriesRequest, Querier_SeriesServer) error + Next(*NextRequest, Querier_NextServer) error +} + +// UnimplementedQuerierServer can be embedded to have forward compatible implementations. +type UnimplementedQuerierServer struct { +} + +func (*UnimplementedQuerierServer) Series(req *SeriesRequest, srv Querier_SeriesServer) error { + return status.Errorf(codes.Unimplemented, "method Series not implemented") +} +func (*UnimplementedQuerierServer) Next(req *NextRequest, srv Querier_NextServer) error { + return status.Errorf(codes.Unimplemented, "method Next not implemented") +} + +func RegisterQuerierServer(s *grpc.Server, srv QuerierServer) { + s.RegisterService(&_Querier_serviceDesc, srv) +} + +func _Querier_Series_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(SeriesRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QuerierServer).Series(m, &querierSeriesServer{stream}) +} + +type Querier_SeriesServer interface { + Send(*SeriesBatch) error + grpc.ServerStream +} + +type querierSeriesServer struct { + grpc.ServerStream +} + +func (x *querierSeriesServer) Send(m *SeriesBatch) error { + return x.ServerStream.SendMsg(m) +} + +func _Querier_Next_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(NextRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QuerierServer).Next(m, &querierNextServer{stream}) +} + +type Querier_NextServer interface { + Send(*StepVectorBatch) error + grpc.ServerStream +} + +type querierNextServer struct { + grpc.ServerStream +} + +func (x *querierNextServer) Send(m *StepVectorBatch) error { + return x.ServerStream.SendMsg(m) +} + +var _Querier_serviceDesc = grpc.ServiceDesc{ + ServiceName: "querierpb.Querier", + HandlerType: (*QuerierServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "Series", + Handler: _Querier_Series_Handler, + ServerStreams: true, + }, + { + StreamName: "Next", + Handler: _Querier_Next_Handler, + ServerStreams: true, + }, + }, + Metadata: "pkg/engine/distributed_execution/querierpb/querier.proto", +} + +func (m *SeriesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Batchsize != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.Batchsize)) + i-- + dAtA[i] = 0x18 + } + if m.FragmentID != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.FragmentID)) + i-- + dAtA[i] = 0x10 + } + if m.QueryID != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.QueryID)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *NextRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *NextRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *NextRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Batchsize != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.Batchsize)) + i-- + dAtA[i] = 0x18 + } + if m.FragmentID != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.FragmentID)) + i-- + dAtA[i] = 0x10 + } + if m.QueryID != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.QueryID)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SeriesBatch) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SeriesBatch) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SeriesBatch) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.OneSeries) > 0 { + for iNdEx := len(m.OneSeries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.OneSeries[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *OneSeries) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *OneSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *OneSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Labels[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *StepVectorBatch) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StepVectorBatch) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StepVectorBatch) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.StepVectors) > 0 { + for iNdEx := len(m.StepVectors) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.StepVectors[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *StepVector) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StepVector) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StepVector) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Histograms) > 0 { + for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + } + if len(m.Histogram_IDs) > 0 { + dAtA2 := make([]byte, len(m.Histogram_IDs)*10) + var j1 int + for _, num := range m.Histogram_IDs { + for num >= 1<<7 { + dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA2[j1] = uint8(num) + j1++ + } + i -= j1 + copy(dAtA[i:], dAtA2[:j1]) + i = encodeVarintQuerier(dAtA, i, uint64(j1)) + i-- + dAtA[i] = 0x22 + } + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + f3 := math.Float64bits(float64(m.Samples[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f3)) + } + i = encodeVarintQuerier(dAtA, i, uint64(len(m.Samples)*8)) + i-- + dAtA[i] = 0x1a + } + if len(m.Sample_IDs) > 0 { + dAtA5 := make([]byte, len(m.Sample_IDs)*10) + var j4 int + for _, num := range m.Sample_IDs { + for num >= 1<<7 { + dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j4++ + } + dAtA5[j4] = uint8(num) + j4++ + } + i -= j4 + copy(dAtA[i:], dAtA5[:j4]) + i = encodeVarintQuerier(dAtA, i, uint64(j4)) + i-- + dAtA[i] = 0x12 + } + if m.T != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.T)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Label) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Label) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Label) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintQuerier(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x12 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintQuerier(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Histogram) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Histogram) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.TimestampMs != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.TimestampMs)) + i-- + dAtA[i] = 0x78 + } + if m.ResetHint != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.ResetHint)) + i-- + dAtA[i] = 0x70 + } + if len(m.PositiveCounts) > 0 { + for iNdEx := len(m.PositiveCounts) - 1; iNdEx >= 0; iNdEx-- { + f6 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) + } + i = encodeVarintQuerier(dAtA, i, uint64(len(m.PositiveCounts)*8)) + i-- + dAtA[i] = 0x6a + } + if len(m.PositiveDeltas) > 0 { + var j7 int + dAtA9 := make([]byte, len(m.PositiveDeltas)*10) + for _, num := range m.PositiveDeltas { + x8 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x8 >= 1<<7 { + dAtA9[j7] = uint8(uint64(x8)&0x7f | 0x80) + j7++ + x8 >>= 7 + } + dAtA9[j7] = uint8(x8) + j7++ + } + i -= j7 + copy(dAtA[i:], dAtA9[:j7]) + i = encodeVarintQuerier(dAtA, i, uint64(j7)) + i-- + dAtA[i] = 0x62 + } + if len(m.PositiveSpans) > 0 { + for iNdEx := len(m.PositiveSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.PositiveSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } + } + if len(m.NegativeCounts) > 0 { + for iNdEx := len(m.NegativeCounts) - 1; iNdEx >= 0; iNdEx-- { + f10 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f10)) + } + i = encodeVarintQuerier(dAtA, i, uint64(len(m.NegativeCounts)*8)) + i-- + dAtA[i] = 0x52 + } + if len(m.NegativeDeltas) > 0 { + var j11 int + dAtA13 := make([]byte, len(m.NegativeDeltas)*10) + for _, num := range m.NegativeDeltas { + x12 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x12 >= 1<<7 { + dAtA13[j11] = uint8(uint64(x12)&0x7f | 0x80) + j11++ + x12 >>= 7 + } + dAtA13[j11] = uint8(x12) + j11++ + } + i -= j11 + copy(dAtA[i:], dAtA13[:j11]) + i = encodeVarintQuerier(dAtA, i, uint64(j11)) + i-- + dAtA[i] = 0x4a + } + if len(m.NegativeSpans) > 0 { + for iNdEx := len(m.NegativeSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NegativeSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuerier(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + if m.ZeroCount != nil { + { + size := m.ZeroCount.Size() + i -= size + if _, err := m.ZeroCount.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.ZeroThreshold != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroThreshold)))) + i-- + dAtA[i] = 0x29 + } + if m.Schema != 0 { + i = encodeVarintQuerier(dAtA, i, uint64((uint32(m.Schema)<<1)^uint32((m.Schema>>31)))) + i-- + dAtA[i] = 0x20 + } + if m.Sum != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Sum)))) + i-- + dAtA[i] = 0x19 + } + if m.Count != nil { + { + size := m.Count.Size() + i -= size + if _, err := m.Count.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Histogram_CountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintQuerier(dAtA, i, uint64(m.CountInt)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *Histogram_CountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.CountFloat)))) + i-- + dAtA[i] = 0x11 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintQuerier(dAtA, i, uint64(m.ZeroCountInt)) + i-- + dAtA[i] = 0x30 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroCountFloat)))) + i-- + dAtA[i] = 0x39 + return len(dAtA) - i, nil +} +func (m *BucketSpan) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BucketSpan) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BucketSpan) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Length != 0 { + i = encodeVarintQuerier(dAtA, i, uint64(m.Length)) + i-- + dAtA[i] = 0x10 + } + if m.Offset != 0 { + i = encodeVarintQuerier(dAtA, i, uint64((uint32(m.Offset)<<1)^uint32((m.Offset>>31)))) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintQuerier(dAtA []byte, offset int, v uint64) int { + offset -= sovQuerier(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *SeriesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.QueryID != 0 { + n += 1 + sovQuerier(uint64(m.QueryID)) + } + if m.FragmentID != 0 { + n += 1 + sovQuerier(uint64(m.FragmentID)) + } + if m.Batchsize != 0 { + n += 1 + sovQuerier(uint64(m.Batchsize)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *NextRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.QueryID != 0 { + n += 1 + sovQuerier(uint64(m.QueryID)) + } + if m.FragmentID != 0 { + n += 1 + sovQuerier(uint64(m.FragmentID)) + } + if m.Batchsize != 0 { + n += 1 + sovQuerier(uint64(m.Batchsize)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *SeriesBatch) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.OneSeries) > 0 { + for _, e := range m.OneSeries { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *OneSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *StepVectorBatch) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.StepVectors) > 0 { + for _, e := range m.StepVectors { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *StepVector) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.T != 0 { + n += 1 + sovQuerier(uint64(m.T)) + } + if len(m.Sample_IDs) > 0 { + l = 0 + for _, e := range m.Sample_IDs { + l += sovQuerier(uint64(e)) + } + n += 1 + sovQuerier(uint64(l)) + l + } + if len(m.Samples) > 0 { + n += 1 + sovQuerier(uint64(len(m.Samples)*8)) + len(m.Samples)*8 + } + if len(m.Histogram_IDs) > 0 { + l = 0 + for _, e := range m.Histogram_IDs { + l += sovQuerier(uint64(e)) + } + n += 1 + sovQuerier(uint64(l)) + l + } + if len(m.Histograms) > 0 { + for _, e := range m.Histograms { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Label) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovQuerier(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovQuerier(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Histogram) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Count != nil { + n += m.Count.Size() + } + if m.Sum != 0 { + n += 9 + } + if m.Schema != 0 { + n += 1 + sozQuerier(uint64(m.Schema)) + } + if m.ZeroThreshold != 0 { + n += 9 + } + if m.ZeroCount != nil { + n += m.ZeroCount.Size() + } + if len(m.NegativeSpans) > 0 { + for _, e := range m.NegativeSpans { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if len(m.NegativeDeltas) > 0 { + l = 0 + for _, e := range m.NegativeDeltas { + l += sozQuerier(uint64(e)) + } + n += 1 + sovQuerier(uint64(l)) + l + } + if len(m.NegativeCounts) > 0 { + n += 1 + sovQuerier(uint64(len(m.NegativeCounts)*8)) + len(m.NegativeCounts)*8 + } + if len(m.PositiveSpans) > 0 { + for _, e := range m.PositiveSpans { + l = e.Size() + n += 1 + l + sovQuerier(uint64(l)) + } + } + if len(m.PositiveDeltas) > 0 { + l = 0 + for _, e := range m.PositiveDeltas { + l += sozQuerier(uint64(e)) + } + n += 1 + sovQuerier(uint64(l)) + l + } + if len(m.PositiveCounts) > 0 { + n += 1 + sovQuerier(uint64(len(m.PositiveCounts)*8)) + len(m.PositiveCounts)*8 + } + if m.ResetHint != 0 { + n += 1 + sovQuerier(uint64(m.ResetHint)) + } + if m.TimestampMs != 0 { + n += 1 + sovQuerier(uint64(m.TimestampMs)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Histogram_CountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovQuerier(uint64(m.CountInt)) + return n +} +func (m *Histogram_CountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *Histogram_ZeroCountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovQuerier(uint64(m.ZeroCountInt)) + return n +} +func (m *Histogram_ZeroCountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *BucketSpan) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Offset != 0 { + n += 1 + sozQuerier(uint64(m.Offset)) + } + if m.Length != 0 { + n += 1 + sovQuerier(uint64(m.Length)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovQuerier(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozQuerier(x uint64) (n int) { + return sovQuerier(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *SeriesRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryID", wireType) + } + m.QueryID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.QueryID |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FragmentID", wireType) + } + m.FragmentID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FragmentID |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Batchsize", wireType) + } + m.Batchsize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Batchsize |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *NextRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: NextRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: NextRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryID", wireType) + } + m.QueryID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.QueryID |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FragmentID", wireType) + } + m.FragmentID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FragmentID |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Batchsize", wireType) + } + m.Batchsize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Batchsize |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SeriesBatch) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SeriesBatch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SeriesBatch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OneSeries", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OneSeries = append(m.OneSeries, &OneSeries{}) + if err := m.OneSeries[len(m.OneSeries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *OneSeries) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: OneSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: OneSeries: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, &Label{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StepVectorBatch) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StepVectorBatch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StepVectorBatch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StepVectors", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StepVectors = append(m.StepVectors, &StepVector{}) + if err := m.StepVectors[len(m.StepVectors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *StepVector) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StepVector: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StepVector: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field T", wireType) + } + m.T = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.T |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Sample_IDs = append(m.Sample_IDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Sample_IDs) == 0 { + m.Sample_IDs = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Sample_IDs = append(m.Sample_IDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Sample_IDs", wireType) + } + case 3: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.Samples = append(m.Samples, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Samples) == 0 { + m.Samples = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.Samples = append(m.Samples, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + } + case 4: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Histogram_IDs = append(m.Histogram_IDs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Histogram_IDs) == 0 { + m.Histogram_IDs = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Histogram_IDs = append(m.Histogram_IDs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Histogram_IDs", wireType) + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histograms", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Histograms = append(m.Histograms, Histogram{}) + if err := m.Histograms[len(m.Histograms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Label) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Label: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Label: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + 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 ErrInvalidLengthQuerier + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + 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 ErrInvalidLengthQuerier + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Histogram) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Histogram: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Histogram: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CountInt", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Count = &Histogram_CountInt{v} + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = &Histogram_CountFloat{float64(math.Float64frombits(v))} + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Sum = float64(math.Float64frombits(v)) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Schema = v + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroThreshold", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroThreshold = float64(math.Float64frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountInt", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ZeroCount = &Histogram_ZeroCountInt{v} + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroCount = &Histogram_ZeroCountFloat{float64(math.Float64frombits(v))} + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NegativeSpans = append(m.NegativeSpans, BucketSpan{}) + if err := m.NegativeSpans[len(m.NegativeSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NegativeDeltas) == 0 { + m.NegativeDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeDeltas", wireType) + } + case 10: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.NegativeCounts) == 0 { + m.NegativeCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeCounts", wireType) + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PositiveSpans = append(m.PositiveSpans, BucketSpan{}) + if err := m.PositiveSpans[len(m.PositiveSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.PositiveDeltas) == 0 { + m.PositiveDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveDeltas", wireType) + } + case 13: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthQuerier + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthQuerier + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.PositiveCounts) == 0 { + m.PositiveCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveCounts", wireType) + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ResetHint", wireType) + } + m.ResetHint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ResetHint |= Histogram_ResetHint(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + } + m.TimestampMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimestampMs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BucketSpan) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BucketSpan: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BucketSpan: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Offset = v + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Length", wireType) + } + m.Length = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuerier + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Length |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipQuerier(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuerier + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipQuerier(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuerier + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuerier + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowQuerier + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthQuerier + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupQuerier + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthQuerier + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthQuerier = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowQuerier = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupQuerier = fmt.Errorf("proto: unexpected end of group") +) diff --git a/pkg/engine/distributed_execution/querierpb/querier.proto b/pkg/engine/distributed_execution/querierpb/querier.proto new file mode 100644 index 00000000000..5deb968d4ec --- /dev/null +++ b/pkg/engine/distributed_execution/querierpb/querier.proto @@ -0,0 +1,118 @@ +syntax = "proto3"; + +package querierpb; + +import "gogoproto/gogo.proto"; +option (gogoproto.marshaler_all) = true; +option (gogoproto.unmarshaler_all) = true; +option (gogoproto.sizer_all) = true; + +option go_package = "./pkg/engine/distributed_execution/querierpb"; + +service Querier { + rpc Series(SeriesRequest) returns (stream SeriesBatch) {} + rpc Next(NextRequest) returns (stream StepVectorBatch) {} +} + +message SeriesRequest { + uint64 queryID = 1; + uint64 fragmentID = 2; + int64 batchsize = 3; +} + +message NextRequest { + uint64 queryID = 1; + uint64 fragmentID = 2; + int64 batchsize = 3; +} + +message SeriesBatch { + repeated OneSeries OneSeries = 1; +} + +message OneSeries { + repeated Label labels = 1; +} + +message StepVectorBatch { + repeated StepVector step_vectors = 1; +} + +message StepVector { + int64 t = 1; + repeated uint64 sample_IDs = 2; + repeated double samples = 3; + repeated uint64 histogram_IDs = 4; + repeated Histogram histograms = 5 [(gogoproto.nullable) = false]; +} + +message Label { + string name = 1; + string value = 2; +} + +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +message Histogram { + enum ResetHint { + option (gogoproto.goproto_enum_prefix) = true; + UNKNOWN = 0; // Need to test for a counter reset explicitly. + YES = 1; // This is the 1st histogram after a counter reset. + NO = 2; // There was no counter reset between this and the previous Histogram. + GAUGE = 3; // This is a gauge histogram where counter resets don't happen. + } + + oneof count { // Count of observations in the histogram. + uint64 count_int = 1; + double count_float = 2; + } + double sum = 3; // Sum of observations in the histogram. + // The schema defines the bucket schema. Currently, valid numbers + // are -4 <= n <= 8. They are all for base-2 bucket schemas, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n logarithmic buckets. Or in other words, each + // bucket boundary is the previous boundary times 2^(2^-n). In the + // future, more bucket schemas may be added using numbers < -4 or > + // 8. + sint32 schema = 4; + double zero_threshold = 5; // Breadth of the zero bucket. + oneof zero_count { // Count in zero bucket. + uint64 zero_count_int = 6; + double zero_count_float = 7; + } + + // Negative Buckets. + repeated BucketSpan negative_spans = 8 [(gogoproto.nullable) = false]; + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 negative_deltas = 9; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double negative_counts = 10; // Absolute count of each bucket. + + // Positive Buckets. + repeated BucketSpan positive_spans = 11 [(gogoproto.nullable) = false]; + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for float + // histograms. + repeated sint64 positive_deltas = 12; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double positive_counts = 13; // Absolute count of each bucket. + + ResetHint reset_hint = 14; + // timestamp is in ms format, see model/timestamp/timestamp.go for + // conversion from time.Time to Prometheus timestamp. + int64 timestamp_ms = 15; +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +message BucketSpan { + sint32 offset = 1; // Gap to previous span, or starting point for 1st span (which can be negative). + uint32 length = 2; // Length of consecutive buckets. +} \ No newline at end of file diff --git a/pkg/engine/distributed_execution/remote_node.go b/pkg/engine/distributed_execution/remote_node.go new file mode 100644 index 00000000000..4ae2b60b51f --- /dev/null +++ b/pkg/engine/distributed_execution/remote_node.go @@ -0,0 +1,371 @@ +package distributed_execution + +import ( + "context" + "encoding/json" + "fmt" + "io" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/storage" + "github.com/thanos-io/promql-engine/execution/exchange" + "github.com/thanos-io/promql-engine/execution/model" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + + "github.com/cortexproject/cortex/pkg/engine/distributed_execution/querierpb" + "github.com/cortexproject/cortex/pkg/ring/client" +) + +type NodeType = logicalplan.NodeType +type Node = logicalplan.Node + +const ( + RemoteNode = "RemoteNode" +) + +// (to verify interface implementations) +var _ logicalplan.Node = (*Remote)(nil) +var _ logicalplan.UserDefinedExpr = (*Remote)(nil) + +type Remote struct { + Op parser.ItemType + Expr Node `json:"-"` + + FragmentKey FragmentKey + FragmentAddr string +} + +func NewRemoteNode() Node { + return &Remote{ + // initialize the fragment key pointer first + FragmentKey: FragmentKey{}, + } +} +func (r *Remote) Clone() Node { + return &Remote{Op: r.Op, Expr: r.Expr.Clone(), FragmentKey: r.FragmentKey} +} +func (r *Remote) Children() []*Node { + return []*Node{&r.Expr} +} +func (r *Remote) String() string { + return fmt.Sprintf("%s%s", r.Op.String(), r.Expr.String()) +} +func (r *Remote) ReturnType() parser.ValueType { + return r.Expr.ReturnType() +} +func (r *Remote) Type() NodeType { return RemoteNode } + +type remote struct { + QueryID uint64 + FragmentID uint64 + FragmentAddr string +} + +func (r *Remote) MarshalJSON() ([]byte, error) { + return json.Marshal(remote{ + QueryID: r.FragmentKey.queryID, + FragmentID: r.FragmentKey.fragmentID, + FragmentAddr: r.FragmentAddr, + }) +} + +func (r *Remote) UnmarshalJSON(data []byte) error { + re := remote{} + if err := json.Unmarshal(data, &re); err != nil { + return err + } + + r.FragmentKey = MakeFragmentKey(re.QueryID, re.FragmentID) + r.FragmentAddr = re.FragmentAddr + return nil +} + +type poolKey struct{} + +// TODO: change to using an extra layer to put the querier address in it +func ContextWithPool(ctx context.Context, pool *client.Pool) context.Context { + return context.WithValue(ctx, poolKey{}, pool) +} + +func PoolFromContext(ctx context.Context) *client.Pool { + if pool, ok := ctx.Value(poolKey{}).(*client.Pool); ok { + return pool + } + return nil +} + +func (p *Remote) MakeExecutionOperator( + ctx context.Context, + vectors *model.VectorPool, + opts *query.Options, + hints storage.SelectHints, +) (model.VectorOperator, error) { + pool := PoolFromContext(ctx) + if pool == nil { + return nil, fmt.Errorf("client pool not found in context") + } + + remoteExec, err := newDistributedRemoteExecution(ctx, pool, p.FragmentKey, opts) + if err != nil { + return nil, err + } + return exchange.NewConcurrent(remoteExec, 2, opts), nil +} + +type DistributedRemoteExecution struct { + client querierpb.QuerierClient + + mint int64 + maxt int64 + step int64 + currentStep int64 + numSteps int + + stream querierpb.Querier_NextClient + buffer []model.StepVector + bufferIndex int + + batchSize int64 + series []labels.Labels + fragmentKey FragmentKey + addr string + initialized bool // Track if stream is initialized +} + +type QuerierAddrKey struct{} + +func newDistributedRemoteExecution(ctx context.Context, pool *client.Pool, fragmentKey FragmentKey, queryOpts *query.Options) (*DistributedRemoteExecution, error) { + + _, _, _, childIDToAddr, _ := ExtractFragmentMetaData(ctx) + + poolClient, err := pool.GetClientFor(childIDToAddr[fragmentKey.fragmentID]) + + if err != nil { + return nil, err + } + + client, ok := poolClient.(*querierClient) + if !ok { + return nil, fmt.Errorf("invalid client type from pool") + } + + d := &DistributedRemoteExecution{ + client: client, + + mint: queryOpts.Start.UnixMilli(), + maxt: queryOpts.End.UnixMilli(), + step: queryOpts.Step.Milliseconds(), + currentStep: queryOpts.Start.UnixMilli(), + numSteps: queryOpts.NumSteps(), + + batchSize: 1000, + fragmentKey: fragmentKey, + addr: childIDToAddr[fragmentKey.fragmentID], + buffer: nil, + bufferIndex: 0, + initialized: false, + } + + if d.step == 0 { + d.step = 1 + } + + return d, nil +} + +func (d *DistributedRemoteExecution) Series(ctx context.Context) ([]labels.Labels, error) { + + if d.series != nil { + return d.series, nil + } + + req := &querierpb.SeriesRequest{ + QueryID: d.fragmentKey.queryID, + FragmentID: d.fragmentKey.fragmentID, + Batchsize: d.batchSize, + } + + stream, err := d.client.Series(ctx, req) + if err != nil { + return nil, err + } + + var series []labels.Labels + + for { + seriesBatch, err := stream.Recv() + if err == io.EOF { + break + } + if err != nil { + return nil, err + } + + for _, s := range seriesBatch.OneSeries { + oneSeries := make([]labels.Label, len(s.Labels)) + for j, l := range s.Labels { + oneSeries[j] = labels.Label{Name: l.Name, Value: l.Value} + } + series = append(series, oneSeries) + } + } + + d.series = series + return series, nil +} + +func (d *DistributedRemoteExecution) Next(ctx context.Context) ([]model.StepVector, error) { + + select { + case <-ctx.Done(): + return nil, ctx.Err() + default: + } + + if d.currentStep > d.maxt { + return nil, nil + } + + ts := d.currentStep + numVectorsNeeded := 0 + for currStep := 0; currStep < d.numSteps && ts <= d.maxt; currStep++ { + numVectorsNeeded++ + ts += d.step + } + + // return from buffer first + if d.buffer != nil && d.bufferIndex < len(d.buffer) { + end := d.bufferIndex + int(d.batchSize) + if end > len(d.buffer) { + end = len(d.buffer) + } + result := d.buffer[d.bufferIndex:end] + d.bufferIndex = end + + if d.bufferIndex >= len(d.buffer) { + d.buffer = nil + d.bufferIndex = 0 + } + + return result, nil + } + + // initialize stream if haven't + if !d.initialized { + req := &querierpb.NextRequest{ + QueryID: d.fragmentKey.queryID, + FragmentID: d.fragmentKey.fragmentID, + Batchsize: d.batchSize, + } + stream, err := d.client.Next(ctx, req) + if err != nil { + return nil, fmt.Errorf("failed to initialize stream: %w", err) + } + d.stream = stream + d.initialized = true + } + + // get new batch from server + batch, err := d.stream.Recv() + if err == io.EOF { + return nil, nil + } + if err != nil { + return nil, fmt.Errorf("error receiving from stream: %w", err) + } + + // return new batch and save it + d.buffer = make([]model.StepVector, len(batch.StepVectors)) + for i, sv := range batch.StepVectors { + d.buffer[i] = model.StepVector{ + T: sv.T, + SampleIDs: sv.Sample_IDs, + Samples: sv.Samples, + HistogramIDs: sv.Histogram_IDs, + Histograms: FloatHistogramProtoToFloatHistograms(sv.Histograms), + } + } + + end := int(d.batchSize) + if end > len(d.buffer) { + end = len(d.buffer) + } + result := d.buffer[:end] + d.bufferIndex = end + + if d.bufferIndex >= len(d.buffer) { + d.buffer = nil + d.bufferIndex = 0 + } + + d.currentStep += d.step * int64(len(result)) + + return result, nil +} + +func (d *DistributedRemoteExecution) Close() error { + if d.stream != nil { + + if err := d.stream.CloseSend(); err != nil { + return fmt.Errorf("error closing stream: %w", err) + } + } + d.buffer = nil + d.bufferIndex = 0 + d.initialized = false + return nil +} + +// +//func (d *DistributedRemoteExecution) Next(ctx context.Context) ([]model.StepVector, error) { +// req := &querierpb.NextRequest{ +// QueryID: d.fragmentKey.queryID, +// FragmentID: d.fragmentKey.fragmentID, +// Batchsize: d.batchSize, +// } +// +// stream, err := d.client.Next(ctx, req) +// if err != nil { +// return nil, err +// } +// +// // init do once Do (func) +// batch, err := stream.Recv() +// if err == io.EOF { +// return nil, nil +// } +// if err != nil { +// return nil, err +// } +// +// result := make([]model.StepVector, len(batch.StepVectors)) +// +// for i, sv := range batch.StepVectors { +// result[i] = model.StepVector{ +// T: sv.T, +// SampleIDs: sv.Sample_IDs, +// Samples: sv.Samples, +// HistogramIDs: sv.Histogram_IDs, +// Histograms: FloatHistogramProtoToFloatHistograms(sv.Histograms), +// } +// } +// +// return result, nil +//} + +func (d DistributedRemoteExecution) GetPool() *model.VectorPool { + //TODO + return &model.VectorPool{} +} + +func (d DistributedRemoteExecution) Explain() (next []model.VectorOperator) { + //TODO + return []model.VectorOperator{} +} + +func (d DistributedRemoteExecution) String() string { + //TODO implement + return "distributed remote execution" +} diff --git a/pkg/engine/distributed_execution/remote_node_test.go b/pkg/engine/distributed_execution/remote_node_test.go new file mode 100644 index 00000000000..e2d8d5962ab --- /dev/null +++ b/pkg/engine/distributed_execution/remote_node_test.go @@ -0,0 +1 @@ +package distributed_execution diff --git a/pkg/engine/distributed_execution/result_cache.go b/pkg/engine/distributed_execution/result_cache.go new file mode 100644 index 00000000000..7a1d8b59038 --- /dev/null +++ b/pkg/engine/distributed_execution/result_cache.go @@ -0,0 +1,134 @@ +package distributed_execution + +import ( + "sync" + "time" +) + +const ( + DefaultTTL = 1 * time.Minute +) + +type QueryResultCache struct { + sync.RWMutex + cache map[FragmentKey]FragmentResult +} + +type FragmentStatus string + +const ( + StatusWriting FragmentStatus = "writing" + StatusDone FragmentStatus = "done" + StatusError FragmentStatus = "error" +) + +type FragmentResult struct { + Data interface{} + Status FragmentStatus + Expiration time.Time +} + +type FragmentKey struct { + queryID uint64 + fragmentID uint64 +} + +func MakeFragmentKey(queryID uint64, fragmentID uint64) FragmentKey { + return FragmentKey{ + queryID: queryID, + fragmentID: fragmentID, + } +} + +func (f FragmentKey) GetQueryID() uint64 { + return f.queryID +} + +func (f FragmentKey) GetFragmentID() uint64 { + return f.fragmentID +} + +func NewQueryResultCache() *QueryResultCache { + return &QueryResultCache{ + cache: make(map[FragmentKey]FragmentResult), + } +} + +func (qrc *QueryResultCache) Size() int { + return len(qrc.cache) +} + +func (qrc *QueryResultCache) InitWriting(key FragmentKey) { + qrc.Lock() + defer qrc.Unlock() + qrc.cache[key] = FragmentResult{ + Status: StatusWriting, + Expiration: time.Now().Add(DefaultTTL), + } +} + +func (qrc *QueryResultCache) SetComplete(key FragmentKey, data interface{}) { + qrc.Lock() + defer qrc.Unlock() + qrc.cache[key] = FragmentResult{ + Data: data, + Status: StatusDone, + Expiration: time.Now().Add(DefaultTTL), + } +} + +func (qrc *QueryResultCache) SetError(key FragmentKey) { + qrc.Lock() + defer qrc.Unlock() + qrc.cache[key] = FragmentResult{ + Status: StatusError, + Expiration: time.Now().Add(DefaultTTL), + } +} + +func (qrc *QueryResultCache) IsReady(key FragmentKey) bool { + qrc.RLock() + defer qrc.RUnlock() + if result, ok := qrc.cache[key]; ok { + return result.Status == StatusDone + } + return false +} + +func (qrc *QueryResultCache) Get(key FragmentKey) (FragmentResult, bool) { + qrc.RLock() + defer qrc.RUnlock() + result, ok := qrc.cache[key] + return result, ok +} + +func (qrc *QueryResultCache) GetFragmentStatus(key FragmentKey) FragmentStatus { + qrc.RLock() + defer qrc.RUnlock() + result, ok := qrc.cache[key] + if !ok { + return FragmentStatus("") + } + return result.Status +} + +func (qrc *QueryResultCache) CleanExpired() { + qrc.Lock() + defer qrc.Unlock() + now := time.Now() + for key, result := range qrc.cache { + if now.After(result.Expiration) { + delete(qrc.cache, key) + } + } +} + +func (qrc *QueryResultCache) ClearQuery(queryID uint64) { + qrc.Lock() + defer qrc.Unlock() + for key := range qrc.cache { + if key.queryID == queryID { + delete(qrc.cache, key) + } + } +} diff --git a/pkg/engine/distributed_execution/result_cache_test.go b/pkg/engine/distributed_execution/result_cache_test.go new file mode 100644 index 00000000000..b01ce343964 --- /dev/null +++ b/pkg/engine/distributed_execution/result_cache_test.go @@ -0,0 +1,144 @@ +package distributed_execution + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestQueryResultCache(t *testing.T) { + t.Run("basic operations", func(t *testing.T) { + cache := NewQueryResultCache() + key := MakeFragmentKey(1, 1) + + // test InitWriting + cache.InitWriting(key) + result, ok := cache.Get(key) + require.True(t, ok) + require.Equal(t, StatusWriting, result.Status) + require.False(t, cache.IsReady(key)) + + // test SetComplete + testData := []string{"test"} + cache.SetComplete(key, testData) + result, ok = cache.Get(key) + require.True(t, ok) + require.Equal(t, StatusDone, result.Status) + require.Equal(t, testData, result.Data) + require.True(t, cache.IsReady(key)) + + // test SetError + cache.SetError(key) + result, ok = cache.Get(key) + require.True(t, ok) + require.Equal(t, StatusError, result.Status) + require.False(t, cache.IsReady(key)) + }) + + t.Run("expiration", func(t *testing.T) { + cache := NewQueryResultCache() + key := MakeFragmentKey(1, 1) + + cache.InitWriting(key) + time.Sleep(DefaultTTL) // the default expiration time + + cache.CleanExpired() + _, ok := cache.Get(key) + require.False(t, ok, "Entry should have been cleaned up") + }) + + t.Run("concurrent operations", func(t *testing.T) { + cache := NewQueryResultCache() + const numGoroutines = 10 + const numOperations = 100 + + var wg sync.WaitGroup + wg.Add(numGoroutines * 4) + + // write + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + key := MakeFragmentKey(uint64(routine), uint64(j)) + cache.SetComplete(key, j) + } + }(i) + } + + // read + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + key := MakeFragmentKey(uint64(routine), uint64(j)) + cache.Get(key) + } + }(i) + } + + // error + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + key := MakeFragmentKey(uint64(routine), uint64(j)) + cache.SetError(key) + } + }(i) + } + + // status checkers + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + key := MakeFragmentKey(uint64(routine), uint64(j)) + cache.GetFragmentStatus(key) + } + }(i) + } + + wg.Wait() + }) + + t.Run("query clearing", func(t *testing.T) { + cache := NewQueryResultCache() + + // add multiple fragments for different queries + cache.SetComplete(MakeFragmentKey(1, 1), "data1") + cache.SetComplete(MakeFragmentKey(1, 2), "data2") + cache.SetComplete(MakeFragmentKey(2, 1), "data3") + + // clear query 1 + cache.ClearQuery(1) + + // check query 1 fragments are gone + _, ok := cache.Get(MakeFragmentKey(1, 1)) + require.False(t, ok) + _, ok = cache.Get(MakeFragmentKey(1, 2)) + require.False(t, ok) + + // check query 2 fragment still exists + result, ok := cache.Get(MakeFragmentKey(2, 1)) + require.True(t, ok) + require.Equal(t, "data3", result.Data) + }) + + t.Run("size tracking", func(t *testing.T) { + cache := NewQueryResultCache() + + require.Equal(t, 0, cache.Size()) + + // add entries + cache.SetComplete(MakeFragmentKey(1, 1), "data1") + cache.SetComplete(MakeFragmentKey(1, 2), "data2") + require.Equal(t, 2, cache.Size()) + + // clear entries + cache.ClearQuery(1) + require.Equal(t, 0, cache.Size()) + }) +} diff --git a/pkg/engine/engine.go b/pkg/engine/engine.go index be22e4573ad..1eb4d7f9f81 100644 --- a/pkg/engine/engine.go +++ b/pkg/engine/engine.go @@ -10,6 +10,7 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" thanosengine "github.com/thanos-io/promql-engine/engine" + "github.com/thanos-io/promql-engine/logicalplan" ) type engineKeyType struct{} @@ -43,6 +44,13 @@ func GetEngineType(ctx context.Context) Type { return None } +type QueryEngine interface { + NewInstantQuery(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, qs string, ts time.Time) (promql.Query, error) + NewRangeQuery(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, qs string, start, end time.Time, interval time.Duration) (promql.Query, error) + MakeInstantQueryFromPlan(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, root logicalplan.Node, ts time.Time, qs string) (promql.Query, error) + MakeRangeQueryFromPlan(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, root logicalplan.Node, start time.Time, end time.Time, interval time.Duration, qs string) (promql.Query, error) +} + type Engine struct { prometheusEngine *promql.Engine thanosEngine *thanosengine.Engine @@ -127,6 +135,53 @@ prom: return qf.prometheusEngine.NewRangeQuery(ctx, q, opts, qs, start, end, interval) } +func (qf *Engine) MakeInstantQueryFromPlan(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, root logicalplan.Node, ts time.Time, qs string) (promql.Query, error) { + if engineType := GetEngineType(ctx); engineType == Prometheus { + qf.engineSwitchQueriesTotal.WithLabelValues(string(Prometheus)).Inc() + } else if engineType == Thanos { + qf.engineSwitchQueriesTotal.WithLabelValues(string(Thanos)).Inc() + } + + if qf.thanosEngine != nil { + res, err := qf.thanosEngine.MakeInstantQueryFromPlan(ctx, q, fromPromQLOpts(opts), root, ts) + if err != nil { + if thanosengine.IsUnimplemented(err) { + // fallback to use prometheus engine + qf.fallbackQueriesTotal.Inc() + goto prom + } + return nil, err + } + return res, nil + } + +prom: + return qf.prometheusEngine.NewInstantQuery(ctx, q, opts, qs, ts) +} + +func (qf *Engine) MakeRangeQueryFromPlan(ctx context.Context, q storage.Queryable, opts promql.QueryOpts, root logicalplan.Node, start time.Time, end time.Time, interval time.Duration, qs string) (promql.Query, error) { + if engineType := GetEngineType(ctx); engineType == Prometheus { + qf.engineSwitchQueriesTotal.WithLabelValues(string(Prometheus)).Inc() + } else if engineType == Thanos { + qf.engineSwitchQueriesTotal.WithLabelValues(string(Thanos)).Inc() + } + if qf.thanosEngine != nil { + res, err := qf.thanosEngine.MakeRangeQueryFromPlan(ctx, q, fromPromQLOpts(opts), root, start, end, interval) + if err != nil { + if thanosengine.IsUnimplemented(err) { + // fallback to use prometheus engine + qf.fallbackQueriesTotal.Inc() + goto prom + } + return nil, err + } + return res, nil + } + +prom: + return qf.prometheusEngine.NewRangeQuery(ctx, q, opts, qs, start, end, interval) +} + func fromPromQLOpts(opts promql.QueryOpts) *thanosengine.QueryOpts { if opts == nil { return &thanosengine.QueryOpts{} diff --git a/pkg/engine/engine_test.go b/pkg/engine/engine_test.go index 7b270e6604a..30caa4828d1 100644 --- a/pkg/engine/engine_test.go +++ b/pkg/engine/engine_test.go @@ -14,8 +14,11 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/promql/promqltest" - "github.com/stretchr/testify/require" "github.com/thanos-io/promql-engine/execution/parse" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + + "github.com/stretchr/testify/require" utillog "github.com/cortexproject/cortex/pkg/util/log" ) @@ -123,3 +126,97 @@ func TestEngine_XFunctions(t *testing.T) { }) } } + +func TestEngine_With_Logical_Plan(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewRegistry() + + now := time.Now() + start := time.Now().Add(-time.Minute * 5) + step := time.Minute + queryable := promqltest.LoadedStorage(t, "") + opts := promql.EngineOpts{ + Logger: utillog.GoKitLogToSlog(log.NewNopLogger()), + Reg: reg, + } + queryEngine := New(opts, ThanosEngineConfig{Enabled: true}, reg) + + range_lp := createTestLogicalPlan(t, start, now, step, "up") + instant_lp := createTestLogicalPlan(t, now, now, 0, "up") + + r := &http.Request{Header: http.Header{}} + r.Header.Set(TypeHeader, string(Thanos)) + ctx = AddEngineTypeToContext(ctx, r) + + // Case 1: Executing logical plan with thanos engine + _, _ = queryEngine.MakeInstantQueryFromPlan(ctx, queryable, nil, instant_lp.Root(), now, "up") + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_thanos_engine_fallback_queries_total Total number of fallback queries due to not implementation in thanos engine + # TYPE cortex_thanos_engine_fallback_queries_total counter + cortex_thanos_engine_fallback_queries_total 0 + `), "cortex_thanos_engine_fallback_queries_total")) + + _, _ = queryEngine.MakeRangeQueryFromPlan(ctx, queryable, nil, range_lp.Root(), start, now, step, "up") + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_thanos_engine_fallback_queries_total Total number of fallback queries due to not implementation in thanos engine + # TYPE cortex_thanos_engine_fallback_queries_total counter + cortex_thanos_engine_fallback_queries_total 0 + `), "cortex_thanos_engine_fallback_queries_total")) + + // Case 2: Logical plan that thanos engine cannot execute (so it will fall back to prometheus engine) + err_range_lp := createTestLogicalPlan(t, start, now, step, "up[10]") + _, _ = queryEngine.MakeRangeQueryFromPlan(ctx, queryable, nil, err_range_lp.Root(), start, now, step, "up") + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_thanos_engine_fallback_queries_total Total number of fallback queries due to not implementation in thanos engine + # TYPE cortex_thanos_engine_fallback_queries_total counter + cortex_thanos_engine_fallback_queries_total 1 + `), "cortex_thanos_engine_fallback_queries_total")) + + // Case 3: executing with prometheus engine + r.Header.Set(TypeHeader, string(Prometheus)) + ctx = AddEngineTypeToContext(ctx, r) + + _, _ = queryEngine.MakeInstantQueryFromPlan(ctx, queryable, nil, instant_lp.Root(), now, "up") + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_engine_switch_queries_total Total number of queries where engine_type is set explicitly + # TYPE cortex_engine_switch_queries_total counter + cortex_engine_switch_queries_total{engine_type="prometheus"} 1 + cortex_engine_switch_queries_total{engine_type="thanos"} 3 + `), "cortex_engine_switch_queries_total")) + + _, _ = queryEngine.MakeRangeQueryFromPlan(ctx, queryable, nil, range_lp.Root(), start, now, step, "up") + require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_engine_switch_queries_total Total number of queries where engine_type is set explicitly + # TYPE cortex_engine_switch_queries_total counter + cortex_engine_switch_queries_total{engine_type="prometheus"} 2 + cortex_engine_switch_queries_total{engine_type="thanos"} 3 + `), "cortex_engine_switch_queries_total")) +} + +func createTestLogicalPlan(t *testing.T, startTime time.Time, endTime time.Time, step time.Duration, q string) logicalplan.Plan { + + qOpts := query.Options{ + Start: startTime, + End: startTime, + Step: 0, + StepsBatch: 10, + LookbackDelta: 0, + EnablePerStepStats: false, + } + + if step != 0 { + qOpts.End = endTime + qOpts.Step = step + } + + expr, err := parser.NewParser(q, parser.WithFunctions(parser.Functions)).ParseExpr() + require.NoError(t, err) + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: false, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + + return logicalPlan +} diff --git a/pkg/frontend/config.go b/pkg/frontend/config.go index a1109f213ad..03dff13980e 100644 --- a/pkg/frontend/config.go +++ b/pkg/frontend/config.go @@ -20,8 +20,7 @@ type CombinedFrontendConfig struct { FrontendV1 v1.Config `yaml:",inline"` FrontendV2 v2.Config `yaml:",inline"` - DownstreamURL string `yaml:"downstream_url"` - DistributedExecEnabled bool `yaml:"distributed_exec_enabled" doc:"hidden"` + DownstreamURL string `yaml:"downstream_url"` } func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { @@ -30,7 +29,6 @@ func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { cfg.FrontendV2.RegisterFlags(f) f.StringVar(&cfg.DownstreamURL, "frontend.downstream-url", "", "URL of downstream Prometheus.") - f.BoolVar(&cfg.DistributedExecEnabled, "frontend.distributed-exec-enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan fragments to downstream components.") } // InitFrontend initializes frontend (either V1 -- without scheduler, or V2 -- with scheduler) or no frontend at diff --git a/pkg/frontend/frontend_test.go b/pkg/frontend/frontend_test.go index 08251a2f2ed..4779bda0cac 100644 --- a/pkg/frontend/frontend_test.go +++ b/pkg/frontend/frontend_test.go @@ -291,7 +291,7 @@ func testFrontend(t *testing.T, config CombinedFrontendConfig, handler http.Hand go grpcServer.Serve(grpcListen) //nolint:errcheck var worker services.Service - worker, err = querier_worker.NewQuerierWorker(workerConfig, httpgrpc_server.NewServer(handler), logger, nil) + worker, err = querier_worker.NewQuerierWorker(workerConfig, httpgrpc_server.NewServer(handler), logger, nil, "") require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), worker)) diff --git a/pkg/frontend/v1/frontend_test.go b/pkg/frontend/v1/frontend_test.go index 7ae0c97c299..30978e3b200 100644 --- a/pkg/frontend/v1/frontend_test.go +++ b/pkg/frontend/v1/frontend_test.go @@ -285,7 +285,7 @@ func testFrontend(t *testing.T, config Config, handler http.Handler, test func(a go grpcServer.Serve(grpcListen) //nolint:errcheck var worker services.Service - worker, err = querier_worker.NewQuerierWorker(workerConfig, httpgrpc_server.NewServer(handler), logger, nil) + worker, err = querier_worker.NewQuerierWorker(workerConfig, httpgrpc_server.NewServer(handler), logger, nil, "") require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), worker)) diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 78548030fba..83d5511607f 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -92,10 +92,11 @@ type Config struct { EnablePromQLExperimentalFunctions bool `yaml:"enable_promql_experimental_functions"` // Query Parquet files if available - EnableParquetQueryable bool `yaml:"enable_parquet_queryable"` - ParquetQueryableShardCacheSize int `yaml:"parquet_queryable_shard_cache_size"` - ParquetQueryableDefaultBlockStore string `yaml:"parquet_queryable_default_block_store"` ParquetQueryableFallbackDisabled bool `yaml:"parquet_queryable_fallback_disabled"` + EnableParquetQueryable bool `yaml:"enable_parquet_queryable" doc:"hidden"` + ParquetQueryableShardCacheSize int `yaml:"parquet_queryable_shard_cache_size" doc:"hidden"` + ParquetQueryableDefaultBlockStore string `yaml:"parquet_queryable_default_block_store" doc:"hidden"` + DistributedExecEnabled bool `yaml:"distributed_exec_enabled" doc:"hidden"` } var ( @@ -144,9 +145,10 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.IgnoreMaxQueryLength, "querier.ignore-max-query-length", false, "If enabled, ignore max query length check at Querier select method. Users can choose to ignore it since the validation can be done before Querier evaluation like at Query Frontend or Ruler.") f.BoolVar(&cfg.EnablePromQLExperimentalFunctions, "querier.enable-promql-experimental-functions", false, "[Experimental] If true, experimental promQL functions are enabled.") f.BoolVar(&cfg.EnableParquetQueryable, "querier.enable-parquet-queryable", false, "[Experimental] If true, querier will try to query the parquet files if available.") + f.BoolVar(&cfg.ParquetQueryableFallbackDisabled, "querier.parquet-queryable-fallback-disabled", false, "[Experimental] Disable Parquet queryable to fallback queries to Store Gateway if the block is not available as Parquet files but available in TSDB. Setting this to true will disable the fallback and users can remove Store Gateway. But need to make sure Parquet files are created before it is queryable.") f.IntVar(&cfg.ParquetQueryableShardCacheSize, "querier.parquet-queryable-shard-cache-size", 512, "[Experimental] Maximum size of the Parquet queryable shard cache. 0 to disable.") f.StringVar(&cfg.ParquetQueryableDefaultBlockStore, "querier.parquet-queryable-default-block-store", string(parquetBlockStore), "[Experimental] Parquet queryable's default block store to query. Valid options are tsdb and parquet. If it is set to tsdb, parquet queryable always fallback to store gateway.") - f.BoolVar(&cfg.ParquetQueryableFallbackDisabled, "querier.parquet-queryable-fallback-disabled", false, "[Experimental] Disable Parquet queryable to fallback queries to Store Gateway if the block is not available as Parquet files but available in TSDB. Setting this to true will disable the fallback and users can remove Store Gateway. But need to make sure Parquet files are created before it is queryable.") + f.BoolVar(&cfg.DistributedExecEnabled, "querier.distributed-exec-enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan plan_fragments to downstream components.") } // Validate the config @@ -202,7 +204,7 @@ func getChunksIteratorFunction(_ Config) chunkIteratorFunc { } // New builds a queryable and promql engine. -func New(cfg Config, limits *validation.Overrides, distributor Distributor, stores []QueryableWithFilter, reg prometheus.Registerer, logger log.Logger, isPartialDataEnabled partialdata.IsCfgEnabledFunc) (storage.SampleAndChunkQueryable, storage.ExemplarQueryable, promql.QueryEngine) { +func New(cfg Config, limits *validation.Overrides, distributor Distributor, stores []QueryableWithFilter, reg prometheus.Registerer, logger log.Logger, isPartialDataEnabled partialdata.IsCfgEnabledFunc) (storage.SampleAndChunkQueryable, storage.ExemplarQueryable, engine.QueryEngine) { iteratorFunc := getChunksIteratorFunction(cfg) distributorQueryable := newDistributorQueryable(distributor, cfg.IngesterMetadataStreaming, cfg.IngesterLabelNamesWithMatchers, iteratorFunc, cfg.QueryIngestersWithin, isPartialDataEnabled, cfg.IngesterQueryMaxAttempts) diff --git a/pkg/querier/tripperware/distributed_optimizer.go b/pkg/querier/tripperware/distributed_optimizer.go new file mode 100644 index 00000000000..b33884942cf --- /dev/null +++ b/pkg/querier/tripperware/distributed_optimizer.go @@ -0,0 +1,39 @@ +package tripperware + +import ( + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" + "github.com/prometheus/prometheus/util/annotations" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" +) + +// This is a simplified implementation. +// Future versions of the distributed optimizer are expected to: +// - Support more complex query patterns. +// - Incorporate diverse optimization strategies. +// - Extend support to node types beyond binary operations. + +type DistributedOptimizer struct{} + +func (d *DistributedOptimizer) Optimize(root logicalplan.Node, opts *query.Options) (logicalplan.Node, annotations.Annotations) { + warns := annotations.New() + + logicalplan.TraverseBottomUp(nil, &root, func(parent, current *logicalplan.Node) bool { + + if (*current).Type() == logicalplan.BinaryNode { + ch := (*current).Children() + + // TODO: add more checks so it's not 1+1 + // meed to be vector for lhs and rhs + + for _, child := range ch { + temp := (*child).Clone() + *child = distributed_execution.NewRemoteNode() + *(*child).Children()[0] = temp + } + } + + return false + }) + return root, *warns +} diff --git a/pkg/querier/tripperware/distributed_optimizer_test.go b/pkg/querier/tripperware/distributed_optimizer_test.go new file mode 100644 index 00000000000..61663507de9 --- /dev/null +++ b/pkg/querier/tripperware/distributed_optimizer_test.go @@ -0,0 +1,192 @@ +package tripperware + +import ( + "context" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "testing" + "time" +) + +func TestDistributedOptimizer(t *testing.T) { + testCases := []struct { + name string + query string + start int64 + end int64 + step time.Duration + expected struct { + childrenCount int + remoteExecCount int // the value of remote nodes depends on how the distributed optimizer is designed + remoteExec bool + result string + } + }{ + { + name: "binary operation with aggregations", + query: "sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])) + sum(rate(node_memory_Active_bytes[5m]))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 2, + remoteExec: true, + result: "dedup(remote(sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) + dedup(remote(sum(rate(node_memory_Active_bytes[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])", + }, + }, + { + name: "binary operation with aggregations", + query: "sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])) + sum(rate(node_memory_Active_bytes[5m]))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 2, + remoteExec: true, + result: "dedup(remote(sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) + dedup(remote(sum(rate(node_memory_Active_bytes[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])", + }, + }, + { + name: "multiple binary operations with aggregations", + query: "sum(rate(http_requests_total{job=\"api\"}[5m])) + sum(rate(http_requests_total{job=\"web\"}[5m])) - sum(rate(http_requests_total{job=\"cache\"}[5m]))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 4, + remoteExec: true, + result: "dedup(remote(dedup(remote(sum(rate(http_requests_total{job=\"api\"}[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) + dedup(remote(sum(rate(http_requests_total{job=\"web\"}[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) - dedup(remote(sum(rate(http_requests_total{job=\"cache\"}[5m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])", + }, + }, + { + name: "aggregation with label replacement", + query: "sum(rate(container_cpu_usage_seconds_total[1m])) by (pod) + sum(rate(container_memory_usage_bytes[1m])) by (pod)", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 2, + remoteExec: true, + result: "dedup(remote(sum by (pod) (rate(container_cpu_usage_seconds_total[1m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) + dedup(remote(sum by (pod) (rate(container_memory_usage_bytes[1m]))) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])", + }, + }, + { + name: "subquery with aggregation", + query: "sum(rate(container_network_transmit_bytes_total[5m:1m]))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 0, + remoteExec: true, + result: "sum(rate(container_network_transmit_bytes_total[5m:1m]))", + }, + }, + { + name: "avg over vector with offset", + query: "avg(rate(node_disk_reads_completed_total[5m] offset 1h))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 0, + remoteExec: true, + result: "avg(rate(node_disk_reads_completed_total[5m] offset 1h))", + }, + }, + { + name: "function applied on binary operation", + query: "rate(http_requests_total[5m]) + rate(http_errors_total[5m]) > bool 0", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 4, + remoteExec: true, + result: "dedup(remote(dedup(remote(rate(http_requests_total[5m])) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) + dedup(remote(rate(http_errors_total[5m])) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC]) > bool dedup(remote(0) [1970-01-01 00:01:40 +0000 UTC, 1970-01-01 00:01:40 +0000 UTC])", + }, + }, + { + name: "aggregation without binary, single child", + query: "sum(rate(process_cpu_seconds_total[5m]))", + start: 100000, + end: 100000, + step: time.Minute, + expected: struct { + childrenCount int + remoteExecCount int + remoteExec bool + result string + }{ + remoteExecCount: 0, + remoteExec: true, + result: "sum(rate(process_cpu_seconds_total[5m]))", + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + req := &PrometheusRequest{ + Start: tc.start, + End: tc.end, + Query: tc.query, + } + + middleware := DistributedQueryMiddleware(tc.step, 5*time.Minute) + handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { + return nil, nil + })) + + _, err := handler.Do(context.Background(), req) + require.NoError(t, err) + require.NotNil(t, req.LogicalPlan, "logical plan should be populated") + + root := req.LogicalPlan.Root() + remoteNodeCount := 0 + logicalplan.TraverseBottomUp(nil, &root, func(parent, current *logicalplan.Node) bool { + if distributed_execution.RemoteNode == (*current).Type() { + remoteNodeCount++ + } + return false + }) + require.Equal(t, tc.expected.remoteExecCount, remoteNodeCount) + }) + } +} diff --git a/pkg/querier/tripperware/distributed_query.go b/pkg/querier/tripperware/distributed_query.go index 02a0692153d..e7c039d31ca 100644 --- a/pkg/querier/tripperware/distributed_query.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -66,8 +66,11 @@ func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, e logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) + dOptimizer := DistributedOptimizer{} + dOptimizedPlanNode, _ := dOptimizer.Optimize(optimizedPlan.Root(), &qOpts) + lp := logicalplan.New(dOptimizedPlanNode, &qOpts, planOpts) - return &optimizedPlan, nil + return &lp, nil } func (d distributedQueryMiddleware) Do(ctx context.Context, r Request) (Response, error) { diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index c8b41b165e1..c866226733c 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -15,6 +15,7 @@ import ( otlog "github.com/opentracing/opentracing-go/log" "github.com/prometheus/common/model" v1 "github.com/prometheus/prometheus/web/api/v1" + "github.com/thanos-io/promql-engine/logicalplan" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/api/queryapi" @@ -23,8 +24,6 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/spanlogger" - - "github.com/thanos-io/promql-engine/logicalplan" ) var ( @@ -208,7 +207,7 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } } - h.Add("Content-Type", "application/json") + h.Add("Content-Type", "application/x-www-form-urlencoded") isSourceRuler := strings.Contains(h.Get("User-Agent"), tripperware.RulerUserAgent) if !isSourceRuler { @@ -216,16 +215,19 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) } - byteBody, err := c.getSerializedBody(promReq) + bodyBytes, err := c.getSerializedBody(promReq) if err != nil { return nil, err } + form := url.Values{} + form.Set("plan", string(bodyBytes)) + formEncoded := form.Encode() req := &http.Request{ Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: io.NopCloser(bytes.NewReader(byteBody)), + Body: io.NopCloser(strings.NewReader(formEncoded)), Header: h, } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 122f0645623..0b1de391f8e 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -212,8 +212,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { require.NoError(t, err) // check request body - body, err := io.ReadAll(req.Body) - require.NoError(t, err) + body := []byte(req.PostFormValue("plan")) if tc.expectEmptyBody { require.Empty(t, body) } else { diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index f0b11db6121..786676846bc 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -196,8 +196,7 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques h.Add(n, v) } } - - h.Add("Content-Type", "application/json") + h.Add("Content-Type", "application/x-www-form-urlencoded") tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) @@ -206,11 +205,15 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques return nil, err } + form := url.Values{} + form.Set("plan", string(bodyBytes)) + formEncoded := form.Encode() + req := &http.Request{ Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: io.NopCloser(bytes.NewReader(bodyBytes)), + Body: io.NopCloser(strings.NewReader(formEncoded)), Header: h, } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index f21eae986df..acf66698c16 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -231,8 +231,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { require.NoError(t, err) // check request body - body, err := io.ReadAll(req.Body) - require.NoError(t, err) + body := []byte(req.PostFormValue("plan")) if tc.expectEmptyBody { require.Empty(t, body) } else { diff --git a/pkg/querier/worker/scheduler_processor.go b/pkg/querier/worker/scheduler_processor.go index 10fd96ab230..2914bfc48ba 100644 --- a/pkg/querier/worker/scheduler_processor.go +++ b/pkg/querier/worker/scheduler_processor.go @@ -3,6 +3,7 @@ package worker import ( "context" "fmt" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "net/http" "time" @@ -31,7 +32,7 @@ import ( "github.com/cortexproject/cortex/pkg/util/services" ) -func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer) (*schedulerProcessor, []services.Service) { +func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer, querierAddress string) (*schedulerProcessor, []services.Service) { p := &schedulerProcessor{ log: log, handler: handler, @@ -47,6 +48,7 @@ func newSchedulerProcessor(cfg Config, handler RequestHandler, log log.Logger, r Help: "Time spend doing requests to frontend.", Buckets: prometheus.ExponentialBuckets(0.001, 4, 6), }, []string{"operation", "status_code"}), + querierAddress: querierAddress, // the current querier's address } frontendClientsGauge := promauto.With(reg).NewGauge(prometheus.GaugeOpts{ @@ -71,8 +73,10 @@ type schedulerProcessor struct { grpcConfig grpcclient.Config maxMessageSize int querierID string + querierAddress string frontendPool *client.Pool + querierPool *client.Pool frontendClientRequestDuration *prometheus.HistogramVec targetHeaders []string @@ -97,7 +101,7 @@ func (sp *schedulerProcessor) processQueriesOnSingleStream(ctx context.Context, for backoff.Ongoing() { c, err := schedulerClient.QuerierLoop(ctx) if err == nil { - err = c.Send(&schedulerpb.QuerierToScheduler{QuerierID: sp.querierID}) + err = c.Send(&schedulerpb.QuerierToScheduler{QuerierID: sp.querierID, QuerierAddress: sp.querierAddress}) } if err != nil { @@ -156,7 +160,19 @@ func (sp *schedulerProcessor) querierLoop(c schedulerpb.SchedulerForQuerier_Quer if request.StatsEnabled { level.Info(logger).Log("msg", "started running request") } - sp.runRequest(ctx, logger, request.QueryID, request.FrontendAddress, request.StatsEnabled, request.HttpRequest) + + if len(request.ChildAddr) != len(request.ChildFragmentID) { + level.Error(logger).Log("mismatch between childIDs length (%d) and childAddr length (%d)", len(request.ChildFragmentID), len(request.ChildAddr)) + } else { + ctx = distributed_execution.InjectFragmentMetaData(ctx, request.FragmentID, request.QueryID, request.IsRoot, request.ChildFragmentID, request.ChildAddr) + } + + // if this is a child fragment, then it doesn't matter + // if this is a parent fragment (with remote nodes), then we need the addresses of the queriers that have the child fragments + + ctx = context.WithValue(ctx, distributed_execution.QuerierAddrKey{}, sp.querierAddress) + + sp.runRequest(ctx, logger, request.QueryID, request.FrontendAddress, request.StatsEnabled, request.HttpRequest, request.IsRoot) if err = ctx.Err(); err != nil { return @@ -170,13 +186,13 @@ func (sp *schedulerProcessor) querierLoop(c schedulerpb.SchedulerForQuerier_Quer } } -func (sp *schedulerProcessor) runRequest(ctx context.Context, logger log.Logger, queryID uint64, frontendAddress string, statsEnabled bool, request *httpgrpc.HTTPRequest) { +func (sp *schedulerProcessor) runRequest(ctx context.Context, logger log.Logger, queryID uint64, frontendAddress string, statsEnabled bool, request *httpgrpc.HTTPRequest, isRoot bool) { var stats *querier_stats.QueryStats if statsEnabled { stats, ctx = querier_stats.ContextWithEmptyStats(ctx) } - response, err := sp.handler.Handle(ctx, request) + response, err := sp.handler.Handle(ctx, request) // inject isRoot to ctx then use r.Context().value() if err != nil { var ok bool response, ok = httpgrpc.HTTPResponseFromError(err) @@ -187,6 +203,10 @@ func (sp *schedulerProcessor) runRequest(ctx context.Context, logger log.Logger, } } } + if !isRoot { + return + } + if statsEnabled { level.Info(logger).Log("msg", "finished request", "status_code", response.Code, "response_size", len(response.GetBody())) } @@ -205,7 +225,6 @@ func (sp *schedulerProcessor) runRequest(ctx context.Context, logger log.Logger, Body: []byte(errMsg), } } - c, err := sp.frontendPool.GetClientFor(frontendAddress) if err == nil { // To prevent querier panic, the panic could happen when the go-routines not-exited diff --git a/pkg/querier/worker/scheduler_processor_test.go b/pkg/querier/worker/scheduler_processor_test.go index c3d2534e441..9931ef9e303 100644 --- a/pkg/querier/worker/scheduler_processor_test.go +++ b/pkg/querier/worker/scheduler_processor_test.go @@ -144,7 +144,7 @@ func Test_ToShowNotPanic_RelatedIssue6599(t *testing.T) { go stat.AddFetchedChunkBytes(10) }).Return(&httpgrpc.HTTPResponse{}, nil) - sp, _ := newSchedulerProcessor(cfg, requestHandler, log.NewNopLogger(), nil) + sp, _ := newSchedulerProcessor(cfg, requestHandler, log.NewNopLogger(), nil, "") schedulerClient := &mockSchedulerForQuerierClient{} schedulerClient.On("QuerierLoop", mock.Anything, mock.Anything).Return(querierLoopClient, nil) diff --git a/pkg/querier/worker/worker.go b/pkg/querier/worker/worker.go index 90e32b7aff5..172f7388814 100644 --- a/pkg/querier/worker/worker.go +++ b/pkg/querier/worker/worker.go @@ -3,6 +3,7 @@ package worker import ( "context" "flag" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "os" "sync" "time" @@ -91,7 +92,8 @@ type querierWorker struct { managers map[string]*processorManager } -func NewQuerierWorker(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer) (services.Service, error) { +func NewQuerierWorker(cfg Config, handler RequestHandler, log log.Logger, reg prometheus.Registerer, querierAddr string, + distributedExecEnabled bool, queryResultCache *distributed_execution.QueryResultCache) (services.Service, error) { if cfg.QuerierID == "" { hostname, err := os.Hostname() if err != nil { @@ -109,7 +111,7 @@ func NewQuerierWorker(cfg Config, handler RequestHandler, log log.Logger, reg pr level.Info(log).Log("msg", "Starting querier worker connected to query-scheduler", "scheduler", cfg.SchedulerAddress) address = cfg.SchedulerAddress - processor, servs = newSchedulerProcessor(cfg, handler, log, reg) + processor, servs = newSchedulerProcessor(cfg, handler, log, reg, querierAddr) case cfg.FrontendAddress != "": level.Info(log).Log("msg", "Starting querier worker connected to query-frontend", "frontend", cfg.FrontendAddress) diff --git a/pkg/scheduler/plan_fragments/fragment_table.go b/pkg/scheduler/plan_fragments/fragment_table.go new file mode 100644 index 00000000000..386ba06f8c1 --- /dev/null +++ b/pkg/scheduler/plan_fragments/fragment_table.go @@ -0,0 +1,107 @@ +package plan_fragments + +import ( + "sync" + "time" + + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" +) + +type fragmentEntry struct { + addr string + createdAt time.Time +} + +type FragmentTable struct { + mappings map[distributed_execution.FragmentKey]*fragmentEntry + mu sync.RWMutex + expiration time.Duration +} + +func NewFragmentTable(expiration time.Duration) *FragmentTable { + ft := &FragmentTable{ + mappings: make(map[distributed_execution.FragmentKey]*fragmentEntry), + expiration: expiration, + } + + go ft.periodicCleanup() + + return ft +} + +func (f *FragmentTable) AddMapping(queryID uint64, fragmentID uint64, addr string) { + f.mu.Lock() + defer f.mu.Unlock() + + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + f.mappings[key] = &fragmentEntry{ + addr: addr, + createdAt: time.Now(), + } +} + +func (f *FragmentTable) GetMapping(queryID uint64, fragmentIDs []uint64) ([]string, bool) { + f.mu.RLock() + defer f.mu.RUnlock() + + addresses := make([]string, 0, len(fragmentIDs)) + + for _, fragmentID := range fragmentIDs { + key := distributed_execution.MakeFragmentKey(queryID, fragmentID) + entry, ok := f.mappings[key] + if !ok { + return nil, false + } + + addresses = append(addresses, entry.addr) + } + + return addresses, true +} + +func (f *FragmentTable) ClearMappings(queryID uint64) { + f.mu.Lock() + defer f.mu.Unlock() + + count := 0 + keysToDelete := make([]distributed_execution.FragmentKey, 0) + for key := range f.mappings { + if key.GetQueryID() == queryID { + keysToDelete = append(keysToDelete, key) + count++ + } + } + + for _, key := range keysToDelete { + delete(f.mappings, key) + } +} + +func (f *FragmentTable) cleanupExpired() { + f.mu.Lock() + defer f.mu.Unlock() + + now := time.Now() + expiredCount := 0 + keysToDelete := make([]distributed_execution.FragmentKey, 0) + + for key, entry := range f.mappings { + if now.Sub(entry.createdAt) > f.expiration { + keysToDelete = append(keysToDelete, key) + expiredCount++ + } + } + + for _, key := range keysToDelete { + delete(f.mappings, key) + } +} + +func (f *FragmentTable) periodicCleanup() { + ticker := time.NewTicker(f.expiration / 2) + defer ticker.Stop() + + for range ticker.C { + f.cleanupExpired() + } +} diff --git a/pkg/scheduler/plan_fragments/fragment_table_test.go b/pkg/scheduler/plan_fragments/fragment_table_test.go new file mode 100644 index 00000000000..4c949869d0a --- /dev/null +++ b/pkg/scheduler/plan_fragments/fragment_table_test.go @@ -0,0 +1,228 @@ +package plan_fragments + +import ( + "fmt" + "github.com/stretchr/testify/require" + "sync" + "sync/atomic" + "testing" + "time" +) + +// This test checks whether the hashtable for fragment-querier mapping gives the expected value +// It also checks if it remains functionable during a multi-thread/concurrent read & write situation + +func TestSchedulerCoordination(t *testing.T) { + t.Run("basic operations", func(t *testing.T) { + table := NewFragmentTable() + table.AddMapping(uint64(0), uint64(1), "localhost:8000") + table.AddMapping(uint64(0), uint64(2), "localhost:8001") + + result, exist := table.GetMapping(uint64(0), []uint64{1, 2}) + require.True(t, exist) + require.Equal(t, []string{"localhost:8000", "localhost:8001"}, result) + + result, exist = table.GetMapping(uint64(0), []uint64{1, 3}) + require.False(t, exist) + require.Empty(t, result) + + result, exist = table.GetMapping(uint64(0), []uint64{1}) + require.True(t, exist) + require.Equal(t, []string{"localhost:8000"}, result) + + table.ClearMappings(uint64(0)) + result, exist = table.GetMapping(uint64(0), []uint64{1}) + require.False(t, exist) + require.Empty(t, result) + }) + + t.Run("concurrent operations", func(t *testing.T) { + table := NewFragmentTable() + const numGoroutines = 10 + const numOperations = 100 + + var wg sync.WaitGroup + wg.Add(numGoroutines * 3) // writers, readers, and clearers + + // writers + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + queryID := uint64(routine) + fragmentID := uint64(j) + addr := fmt.Sprintf("localhost:%d", j) + table.AddMapping(queryID, fragmentID, addr) + } + }(i) + } + + // readers + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + queryID := uint64(routine) + fragmentIDs := []uint64{uint64(j)} + table.GetMapping(queryID, fragmentIDs) + } + }(i) + } + + // clearers + for i := 0; i < numGoroutines; i++ { + go func(routine int) { + defer wg.Done() + for j := 0; j < numOperations; j++ { + queryID := uint64(routine) + table.ClearMappings(queryID) + } + }(i) + } + + wg.Wait() + }) + + t.Run("edge cases", func(t *testing.T) { + table := NewFragmentTable() + + // test empty fragment IDs + result, exist := table.GetMapping(0, []uint64{}) + require.True(t, exist) + require.Empty(t, result) + + // test clearing non-existent query + table.ClearMappings(999) + require.NotPanics(t, func() { + table.ClearMappings(999) + }) + + // test overwriting mapping + table.AddMapping(1, 1, "addr1") + table.AddMapping(1, 1, "addr2") + result, exist = table.GetMapping(1, []uint64{1}) + require.True(t, exist) + require.Equal(t, []string{"addr2"}, result) + + // test multiple queries + table.AddMapping(1, 1, "addr1") + table.AddMapping(2, 1, "addr2") + result, exist = table.GetMapping(1, []uint64{1}) + require.True(t, exist) + require.Equal(t, []string{"addr1"}, result) + + result, exist = table.GetMapping(2, []uint64{1}) + require.True(t, exist) + require.Equal(t, []string{"addr2"}, result) + }) +} + +func TestFragmentTableStress(t *testing.T) { + table := NewFragmentTable() + done := make(chan bool) + const duration = 2 * time.Second + + // add counters to check the number of times each + // components are actually executed + var ( + addCount atomic.Uint64 + readCount atomic.Uint64 + clearCount atomic.Uint64 + errorCount atomic.Uint64 + ) + + go func() { + i := uint64(0) + for { + select { + case <-done: + return + default: + func() { + defer func() { + if r := recover(); r != nil { + errorCount.Add(1) + } + }() + table.AddMapping(i, i, fmt.Sprintf("addr%d", i)) + addCount.Add(1) + i++ + }() + } + } + }() + + go func() { + i := uint64(0) + for { + select { + case <-done: + return + default: + func() { + defer func() { + if r := recover(); r != nil { + errorCount.Add(1) + } + }() + _, _ = table.GetMapping(i, []uint64{i}) + readCount.Add(1) + i++ + }() + } + } + }() + + go func() { + i := uint64(0) + for { + select { + case <-done: + return + default: + func() { + defer func() { + if r := recover(); r != nil { + errorCount.Add(1) + } + }() + table.ClearMappings(i) + clearCount.Add(1) + i++ + }() + } + } + }() + + // progress monitoring + ticker := time.NewTicker(200 * time.Millisecond) + defer ticker.Stop() + + go func() { + for { + select { + case <-done: + return + case <-ticker.C: + t.Logf("Progress - Adds: %d, Reads: %d, Clears: %d, Errors: %d", + addCount.Load(), + readCount.Load(), + clearCount.Load(), + errorCount.Load(), + ) + } + } + }() + + time.Sleep(duration) + close(done) + + t.Logf("Final counts - Adds: %d, Reads: %d, Clears: %d, Errors: %d", + addCount.Load(), + readCount.Load(), + clearCount.Load(), + errorCount.Load(), + ) + + require.Equal(t, uint64(0), errorCount.Load(), "stress test produced errors") +} diff --git a/pkg/scheduler/plan_fragments/fragmenter.go b/pkg/scheduler/plan_fragments/fragmenter.go new file mode 100644 index 00000000000..c50971f98ff --- /dev/null +++ b/pkg/scheduler/plan_fragments/fragmenter.go @@ -0,0 +1,85 @@ +package plan_fragments + +import ( + "encoding/binary" + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" + "github.com/google/uuid" + "github.com/thanos-io/promql-engine/logicalplan" +) + +type Fragment struct { + Node logicalplan.Node + FragmentID uint64 + ChildIDs []uint64 + IsRoot bool +} + +func getNewID() uint64 { + id := uuid.New() + return binary.BigEndian.Uint64(id[:8]) +} + +func (s *Fragment) IsEmpty() bool { + if s.Node != nil { + return false + } + if s.FragmentID != 0 { + return false + } + if s.IsRoot { + return false + } + if len(s.ChildIDs) != 0 { + return false + } + return true +} + +// FragmentLogicalPlanNode fragment the logical plan by the remote node +// and inserts the child fragment information into it +func FragmentLogicalPlanNode(queryID uint64, node logicalplan.Node) ([]Fragment, error) { + newFragment := Fragment{} + fragments := []Fragment{} + nextChildrenIDs := []uint64{} + + logicalplan.TraverseBottomUp(nil, &node, func(parent, current *logicalplan.Node) bool { + if parent == nil { // if we have reached the root + newFragment = Fragment{ + Node: node, + FragmentID: getNewID(), + ChildIDs: nextChildrenIDs, + IsRoot: true, + } + fragments = append(fragments, newFragment) + return false // break the loop + } + if distributed_execution.RemoteNode == (*parent).Type() { + newFragment = Fragment{ + Node: *current, + FragmentID: getNewID(), + ChildIDs: []uint64{}, + IsRoot: false, + } + fragments = append(fragments, newFragment) + nextChildrenIDs = append(nextChildrenIDs, newFragment.FragmentID) + + // append remote node information that will be used in the execution stage + key := distributed_execution.MakeFragmentKey(queryID, newFragment.FragmentID) + (*parent).(*distributed_execution.Remote).FragmentKey = key + } + return false + }) + + if fragments != nil { + return fragments, nil + } else { + // for non-query API calls + // --> treat as root fragment and immediately return the result + return []Fragment{{ + Node: node, + FragmentID: uint64(0), + ChildIDs: []uint64{}, + IsRoot: true, + }}, nil + } +} diff --git a/pkg/scheduler/plan_fragments/fragmenter_test.go b/pkg/scheduler/plan_fragments/fragmenter_test.go new file mode 100644 index 00000000000..92f195671a4 --- /dev/null +++ b/pkg/scheduler/plan_fragments/fragmenter_test.go @@ -0,0 +1,60 @@ +package plan_fragments + +import ( + "testing" + "time" + + "github.com/prometheus/prometheus/promql/parser" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" +) + +// This test makes sure that the fragmenter fragments logical plan into the correct number of sub-plans +// (the number of fragments also depends on the distributed optimizer +// , so if it changes the expected value will also need to be adjusted) + +func TestFragmenter(t *testing.T) { + lp := createTestLogicalPlan(t, time.Now(), time.Now(), 0, "sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])) + sum(rate(node_memory_Active_bytes[5m]))") + res, err := FragmentLogicalPlanNode(lp.Root()) + require.NoError(t, err) + require.Equal(t, 3, len(res)) + + lp2 := createTestLogicalPlan(t, time.Now(), time.Now(), 0, "sum(rate(http_requests_total{job=\"api\"}[5m])) + sum(rate(http_requests_total{job=\"web\"}[5m])) - sum(rate(http_requests_total{job=\"cache\"}[5m]))") + res2, err2 := FragmentLogicalPlanNode(lp2.Root()) + require.NoError(t, err2) + require.Equal(t, 5, len(res2)) +} + +func createTestLogicalPlan(t *testing.T, startTime time.Time, endTime time.Time, step time.Duration, q string) logicalplan.Plan { + qOpts := query.Options{ + Start: startTime, + End: startTime, + Step: 0, + StepsBatch: 10, + LookbackDelta: 0, + EnablePerStepStats: false, + } + + if step != 0 { + qOpts.End = endTime + qOpts.Step = step + } + + expr, err := parser.NewParser(q, parser.WithFunctions(parser.Functions)).ParseExpr() + require.NoError(t, err) + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: false, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) + dOptimizer := tripperware.DistributedOptimizer{} + dOptimizedPlanNode, _ := dOptimizer.Optimize(optimizedPlan.Root(), &qOpts) + lp := logicalplan.New(dOptimizedPlanNode, &qOpts, planOpts) + + return lp +} diff --git a/pkg/scheduler/scheduler.go b/pkg/scheduler/scheduler.go index 7c7ef4b7b3e..008a459cb55 100644 --- a/pkg/scheduler/scheduler.go +++ b/pkg/scheduler/scheduler.go @@ -3,8 +3,10 @@ package scheduler import ( "context" "flag" + "fmt" "io" "net/http" + "net/url" "sync" "time" @@ -15,6 +17,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/thanos-io/promql-engine/logicalplan" "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/user" @@ -22,7 +25,9 @@ import ( "github.com/cortexproject/cortex/pkg/frontend/v2/frontendv2pb" //lint:ignore faillint scheduler needs to retrieve priority from the context + "github.com/cortexproject/cortex/pkg/engine/distributed_execution" "github.com/cortexproject/cortex/pkg/querier/stats" + "github.com/cortexproject/cortex/pkg/scheduler/plan_fragments" "github.com/cortexproject/cortex/pkg/scheduler/queue" "github.com/cortexproject/cortex/pkg/scheduler/schedulerpb" "github.com/cortexproject/cortex/pkg/tenant" @@ -39,6 +44,11 @@ var ( errSchedulerIsNotRunning = errors.New("scheduler is not running") ) +const ( + maxRetries = 3 + retryDelay = 100 * time.Millisecond +) + // Scheduler is responsible for queueing and dispatching queries to Queriers. type Scheduler struct { services.Service @@ -55,7 +65,7 @@ type Scheduler struct { activeUsers *util.ActiveUsersCleanupService pendingRequestsMu sync.Mutex - pendingRequests map[requestKey]*schedulerRequest // Request is kept in this map even after being dispatched to querier. It can still be canceled at that time. + pendingRequests map[RequestKey]*schedulerRequest // Request is kept in this map even after being dispatched to querier. It can still be canceled at that time. // Subservices manager. subservices *services.Manager @@ -67,11 +77,16 @@ type Scheduler struct { connectedQuerierClients prometheus.GaugeFunc connectedFrontendClients prometheus.GaugeFunc queueDuration prometheus.Histogram + + // Distributed sub-query mappings + fragmentTable *plan_fragments.FragmentTable + distributedExecEnabled bool } -type requestKey struct { +type RequestKey struct { frontendAddr string queryID uint64 + fragmentID uint64 } type connectedFrontend struct { @@ -95,14 +110,15 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { } // NewScheduler creates a new Scheduler. -func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer prometheus.Registerer) (*Scheduler, error) { +func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer prometheus.Registerer, distributedExecEnabled bool) (*Scheduler, error) { s := &Scheduler{ cfg: cfg, log: log, limits: limits, - pendingRequests: map[requestKey]*schedulerRequest{}, + pendingRequests: map[RequestKey]*schedulerRequest{}, connectedFrontends: map[string]*connectedFrontend{}, + fragmentTable: plan_fragments.NewFragmentTable(2 * time.Minute), } s.queueLength = promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ @@ -139,6 +155,8 @@ func NewScheduler(cfg Config, limits Limits, log log.Logger, registerer promethe return nil, err } + s.distributedExecEnabled = distributedExecEnabled + s.Service = services.NewBasicService(s.starting, s.running, s.stopping) return s, nil } @@ -166,6 +184,8 @@ type schedulerRequest struct { // This is only used for testing. parentSpanContext opentracing.SpanContext + + fragment plan_fragments.Fragment } func (s schedulerRequest) Priority() int64 { @@ -223,7 +243,8 @@ func (s *Scheduler) FrontendLoop(frontend schedulerpb.SchedulerForFrontend_Front } case schedulerpb.CANCEL: - s.cancelRequestAndRemoveFromPending(frontendAddress, msg.QueryID) + // TODO: potential problem... need to move fragmentation step up here maybe + s.cancelRequestAndRemoveFromPending(frontendAddress, msg.QueryID, plan_fragments.Fragment{}) resp = &schedulerpb.SchedulerToFrontend{Status: schedulerpb.OK} default: @@ -279,68 +300,202 @@ func (s *Scheduler) frontendDisconnected(frontendAddress string) { } } -func (s *Scheduler) enqueueRequest(frontendContext context.Context, frontendAddr string, msg *schedulerpb.FrontendToScheduler) error { - // Create new context for this request, to support cancellation. - ctx, cancel := context.WithCancel(frontendContext) - shouldCancel := true - defer func() { - if shouldCancel { - cancel() - } - }() +// fragmentLogicalPlan plan_fragments logical plan if it exist and can be un-marshaled +// otherwise, it returns an empty fragment array +func (s *Scheduler) fragmentLogicalPlan(queryID uint64, req *httpgrpc.HTTPRequest) ([]plan_fragments.Fragment, error) { + if req.Body == nil { + return nil, nil + } + byteLP, err := s.getPlanFromHTTPRequest(req) + if err != nil || len(byteLP) == 0 { + return nil, err + } - // Extract tracing information from headers in HTTP request. FrontendContext doesn't have the correct tracing - // information, since that is a long-running request. - tracer := opentracing.GlobalTracer() - parentSpanContext, err := httpgrpcutil.GetParentSpanForRequest(tracer, msg.HttpRequest) + // distributed execution enabled <--> logical plan exists + lpNode, err := distributed_execution.Unmarshal(byteLP) if err != nil { - return err + return nil, fmt.Errorf("failed to unmarshal logical plan: %w", err) } - userID := msg.GetUserID() + fragments, err := plan_fragments.FragmentLogicalPlanNode(queryID, lpNode) + if err != nil { + return nil, fmt.Errorf("failed to fragment logical plan: %w", err) + } - req := &schedulerRequest{ - frontendAddress: frontendAddr, - userID: msg.UserID, - queryID: msg.QueryID, - request: msg.HttpRequest, - statsEnabled: msg.StatsEnabled, + return fragments, nil +} + +func (s *Scheduler) updatePlanInHTTPRequest(fragment plan_fragments.Fragment) ([]byte, error) { + byteLP, err := logicalplan.Marshal(fragment.Node) + if err != nil { + return nil, err } + form := url.Values{} + form.Add("plan", string(byteLP)) + return []byte(form.Encode()), nil +} - now := time.Now() +func (s *Scheduler) getPlanFromHTTPRequest(req *httpgrpc.HTTPRequest) ([]byte, error) { + values, err := url.ParseQuery(string(req.Body)) + if err != nil { + return nil, fmt.Errorf("failed to parse serialized logical plan: %w", err) + } + plan := values.Get("plan") + return []byte(plan), nil +} - req.parentSpanContext = parentSpanContext - req.queueSpan, req.ctx = opentracing.StartSpanFromContextWithTracer(ctx, tracer, "queued", opentracing.ChildOf(parentSpanContext)) - req.enqueueTime = now - req.ctxCancel = cancel +func (s *Scheduler) enqueueRequest(frontendContext context.Context, frontendAddr string, msg *schedulerpb.FrontendToScheduler) error { + userID := msg.GetUserID() - // aggregate the max queriers limit in the case of a multi tenant query - tenantIDs, err := tenant.TenantIDsFromOrgID(userID) + fragments, err := s.fragmentLogicalPlan(msg.QueryID, msg.HttpRequest) if err != nil { return err } - maxQueriers := validation.SmallestPositiveNonZeroFloat64PerTenant(tenantIDs, s.limits.MaxQueriersPerUser) + if fragments == nil { + + // Create new context for this request, to support cancellation. + ctx, cancel := context.WithCancel(frontendContext) + shouldCancel := true + defer func() { + if shouldCancel { + cancel() + } + }() - s.activeUsers.UpdateUserTimestamp(userID, now) - return s.requestQueue.EnqueueRequest(userID, req, maxQueriers, func() { - shouldCancel = false + // Extract tracing information from headers in HTTP request. FrontendContext doesn't have the correct tracing + // information, since that is a long-running request. + tracer := opentracing.GlobalTracer() + parentSpanContext, err := httpgrpcutil.GetParentSpanForRequest(tracer, msg.HttpRequest) + if err != nil { + return err + } - s.pendingRequestsMu.Lock() - defer s.pendingRequestsMu.Unlock() - s.pendingRequests[requestKey{frontendAddr: frontendAddr, queryID: msg.QueryID}] = req - }) + req := &schedulerRequest{ + frontendAddress: frontendAddr, + userID: msg.UserID, + queryID: msg.QueryID, + request: msg.HttpRequest, + statsEnabled: msg.StatsEnabled, + fragment: plan_fragments.Fragment{ + IsRoot: true, + }, + } + + now := time.Now() + + req.parentSpanContext = parentSpanContext + req.queueSpan, req.ctx = opentracing.StartSpanFromContextWithTracer(ctx, tracer, "queued", opentracing.ChildOf(parentSpanContext)) + req.enqueueTime = now + req.ctxCancel = cancel + + // aggregate the max queriers limit in the case of a multi tenant query + tenantIDs, err := tenant.TenantIDsFromOrgID(userID) + if err != nil { + return err + } + maxQueriers := validation.SmallestPositiveNonZeroFloat64PerTenant(tenantIDs, s.limits.MaxQueriersPerUser) + + s.activeUsers.UpdateUserTimestamp(userID, now) + return s.requestQueue.EnqueueRequest(userID, req, maxQueriers, func() { + shouldCancel = false + + s.pendingRequestsMu.Lock() + defer s.pendingRequestsMu.Unlock() + s.pendingRequests[RequestKey{frontendAddr: frontendAddr, queryID: msg.QueryID, fragmentID: 0}] = req + }) + + } else { // when there are plan_fragments + for _, fragment := range fragments { + + frag := fragment + + if err := func() error { + // create new context and cancel func per fragment + ctx, cancel := context.WithCancel(frontendContext) + shouldCancel := true + defer func() { + if shouldCancel { + cancel() + } + }() + + // extract tracing info + tracer := opentracing.GlobalTracer() + parentSpanContext, err := httpgrpcutil.GetParentSpanForRequest(tracer, msg.HttpRequest) + if err != nil { + return err + } + + // modify request with fragment info + newBody, err := s.updatePlanInHTTPRequest(frag) + msg.HttpRequest = &httpgrpc.HTTPRequest{ + Method: msg.HttpRequest.Method, + Url: msg.HttpRequest.Url, + Headers: msg.HttpRequest.Headers, + Body: newBody, + } + if err != nil { + return err + } + + req := &schedulerRequest{ + frontendAddress: frontendAddr, + userID: msg.UserID, + queryID: msg.QueryID, + request: msg.HttpRequest, + statsEnabled: msg.StatsEnabled, + fragment: frag, + } + + now := time.Now() + req.parentSpanContext = parentSpanContext + req.queueSpan, req.ctx = opentracing.StartSpanFromContextWithTracer(ctx, tracer, "queued", opentracing.ChildOf(parentSpanContext)) + req.enqueueTime = now + req.ctxCancel = cancel + + tenantIDs, err := tenant.TenantIDsFromOrgID(userID) + if err != nil { + return err + } + maxQueriers := validation.SmallestPositiveNonZeroFloat64PerTenant(tenantIDs, s.limits.MaxQueriersPerUser) + + s.activeUsers.UpdateUserTimestamp(userID, now) + + err = s.requestQueue.EnqueueRequest(userID, req, maxQueriers, func() { + shouldCancel = false + s.pendingRequestsMu.Lock() + defer s.pendingRequestsMu.Unlock() + s.pendingRequests[RequestKey{ + frontendAddr: frontendAddr, + queryID: msg.QueryID, + fragmentID: req.fragment.FragmentID, + }] = req + }) + + if err != nil { + return err + } + + return nil + }(); err != nil { + return err + } + } + return nil + } } // This method doesn't do removal from the queue. -func (s *Scheduler) cancelRequestAndRemoveFromPending(frontendAddr string, queryID uint64) { +func (s *Scheduler) cancelRequestAndRemoveFromPending(frontendAddr string, queryID uint64, fragment plan_fragments.Fragment) { s.pendingRequestsMu.Lock() defer s.pendingRequestsMu.Unlock() - key := requestKey{frontendAddr: frontendAddr, queryID: queryID} + key := RequestKey{frontendAddr: frontendAddr, queryID: queryID, fragmentID: fragment.FragmentID} req := s.pendingRequests[key] if req != nil { req.ctxCancel() } + // s.fragmentTable.ClearMappings(queryID) delete(s.pendingRequests, key) } @@ -352,6 +507,7 @@ func (s *Scheduler) QuerierLoop(querier schedulerpb.SchedulerForQuerier_QuerierL } querierID := resp.GetQuerierID() + querierAddress := resp.GetQuerierAddress() s.requestQueue.RegisterQuerierConnection(querierID) defer s.requestQueue.UnregisterQuerierConnection(querierID) @@ -391,22 +547,43 @@ func (s *Scheduler) QuerierLoop(querier schedulerpb.SchedulerForQuerier_QuerierL it's possible that it's own queue would perpetually contain only expired requests. */ - if r.ctx.Err() != nil { + err = r.ctx.Err() + if err != nil { // Remove from pending requests. - s.cancelRequestAndRemoveFromPending(r.frontendAddress, r.queryID) + s.cancelRequestAndRemoveFromPending(r.frontendAddress, r.queryID, r.fragment) lastUserIndex = lastUserIndex.ReuseLastUser() continue } - if err := s.forwardRequestToQuerier(querier, r); err != nil { + if r.fragment.IsRoot { + ok := s.waitForChildToBeReady(r.queryID, r.fragment.ChildIDs) + if ok != true { + return fmt.Errorf("failed to get child address mappings after %d attempts for queryID: %d", maxRetries, r.queryID) + } + } + + if err := s.forwardRequestToQuerier(querier, r, querierAddress); err != nil { return err } } - return errSchedulerIsNotRunning } +func (s *Scheduler) waitForChildToBeReady(queryID uint64, childIDs []uint64) bool { + for attempt := 1; attempt <= maxRetries; attempt++ { + _, ok := s.fragmentTable.GetMapping(queryID, childIDs) + if ok { + return true + } + if attempt == maxRetries { + return false + } + time.Sleep(retryDelay) + } + return false +} + func (s *Scheduler) NotifyQuerierShutdown(_ context.Context, req *schedulerpb.NotifyQuerierShutdownRequest) (*schedulerpb.NotifyQuerierShutdownResponse, error) { level.Info(s.log).Log("msg", "received shutdown notification from querier", "querier", req.GetQuerierID()) s.requestQueue.NotifyQuerierShutdown(req.GetQuerierID()) @@ -414,26 +591,39 @@ func (s *Scheduler) NotifyQuerierShutdown(_ context.Context, req *schedulerpb.No return &schedulerpb.NotifyQuerierShutdownResponse{}, nil } -func (s *Scheduler) forwardRequestToQuerier(querier schedulerpb.SchedulerForQuerier_QuerierLoopServer, req *schedulerRequest) error { +func (s *Scheduler) forwardRequestToQuerier(querier schedulerpb.SchedulerForQuerier_QuerierLoopServer, req *schedulerRequest, querierAddress string) error { // Make sure to cancel request at the end to cleanup resources. - defer s.cancelRequestAndRemoveFromPending(req.frontendAddress, req.queryID) + defer s.cancelRequestAndRemoveFromPending(req.frontendAddress, req.queryID, req.fragment) // Handle the stream sending & receiving on a goroutine so we can // monitoring the contexts in a select and cancel things appropriately. errCh := make(chan error, 1) go func() { + childAddrs := []string{} + if len(req.fragment.ChildIDs) != 0 { + childAddrs, _ = s.fragmentTable.GetMapping(req.queryID, req.fragment.ChildIDs) + } + err := querier.Send(&schedulerpb.SchedulerToQuerier{ UserID: req.userID, QueryID: req.queryID, FrontendAddress: req.frontendAddress, HttpRequest: req.request, StatsEnabled: req.statsEnabled, + FragmentID: req.fragment.FragmentID, + ChildFragmentID: req.fragment.ChildIDs, + ChildAddr: childAddrs, + IsRoot: req.fragment.IsRoot, }) + + if s.distributedExecEnabled { + s.fragmentTable.AddMapping(req.queryID, req.fragment.FragmentID, querierAddress) + } + if err != nil { errCh <- err return } - _, err = querier.Recv() errCh <- err }() diff --git a/pkg/scheduler/scheduler_test.go b/pkg/scheduler/scheduler_test.go index 9c1d75ad51a..e48eddb3ac8 100644 --- a/pkg/scheduler/scheduler_test.go +++ b/pkg/scheduler/scheduler_test.go @@ -3,8 +3,10 @@ package scheduler import ( "context" "fmt" + "github.com/cortexproject/cortex/pkg/querier/tripperware" "net" "net/http" + "net/url" "strings" "sync" "testing" @@ -14,7 +16,10 @@ import ( "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/promql/parser" "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" "github.com/uber/jaeger-client-go/config" "github.com/weaveworks/common/httpgrpc" "google.golang.org/grpc" @@ -35,7 +40,7 @@ const testMaxOutstandingPerTenant = 5 func setupScheduler(t *testing.T, reg prometheus.Registerer) (*Scheduler, schedulerpb.SchedulerForFrontendClient, schedulerpb.SchedulerForQuerierClient) { cfg := Config{} flagext.DefaultValues(&cfg) - s, err := NewScheduler(cfg, frontendv1.MockLimits{Queriers: 2, MockLimits: queue.MockLimits{MaxOutstanding: testMaxOutstandingPerTenant}}, log.NewNopLogger(), reg) + s, err := NewScheduler(cfg, frontendv1.MockLimits{Queriers: 2, MockLimits: queue.MockLimits{MaxOutstanding: testMaxOutstandingPerTenant}}, log.NewNopLogger(), reg, false) require.NoError(t, err) server := grpc.NewServer() @@ -448,6 +453,210 @@ func TestSchedulerMetrics(t *testing.T) { `), "cortex_query_scheduler_queue_length", "cortex_request_queue_requests_total")) } +// TestQuerierLoopClient_WithLogicalPlan tests to see if the scheduler enqueues the fragment +// with the expected QueryID, logical plan, and other fragment meta-data +func TestQuerierLoopClient_WithLogicalPlan(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + + scheduler, frontendClient, querierClient := setupScheduler(t, reg) + frontendLoop := initFrontendLoop(t, frontendClient, "frontend-12345") + querierLoop, err := querierClient.QuerierLoop(context.Background()) + require.NoError(t, err) + + // CASE 1: request with corrupted logical plan --> expect to fail at un-marshal stage + require.NoError(t, frontendLoop.Send(&schedulerpb.FrontendToScheduler{ + Type: schedulerpb.ENQUEUE, + QueryID: 1, + UserID: "test", + HttpRequest: &httpgrpc.HTTPRequest{Method: "POST", Url: "/hello", Body: []byte("test")}, + })) + msg, err := frontendLoop.Recv() + require.NoError(t, err) + require.True(t, msg.Status == schedulerpb.ERROR) + + // CASE 2: request without logical plan --> expect to not have fragment meta-data + frontendToScheduler(t, frontendLoop, &schedulerpb.FrontendToScheduler{ + Type: schedulerpb.ENQUEUE, + QueryID: 2, + UserID: "test2", + HttpRequest: &httpgrpc.HTTPRequest{Method: "POST", Url: "/hello", Body: []byte{}}, // empty logical plan + }) + require.NoError(t, promtest.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_query_scheduler_queue_length Number of queries in the queue. + # TYPE cortex_query_scheduler_queue_length gauge + cortex_query_scheduler_queue_length{priority="0",type="fifo",user="test2"} 1 + # HELP cortex_request_queue_requests_total Total number of query requests going to the request queue. + # TYPE cortex_request_queue_requests_total counter + cortex_request_queue_requests_total{priority="0",user="test2"} 1 + `), "cortex_query_scheduler_queue_length", "cortex_request_queue_requests_total")) + + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{QuerierID: "querier-1", QuerierAddress: "localhost:8000"})) + + s2, err := querierLoop.Recv() + require.NoError(t, err) + require.Equal(t, uint64(2), s2.QueryID) + // (the below fields should be empty because the logical plan is not in the request) + require.Empty(t, s2.FragmentID) + require.Empty(t, s2.ChildFragmentID) + require.Empty(t, s2.ChildAddr) + require.Empty(t, s2.HttpRequest.Body) + require.False(t, s2.IsRoot) + + // CASE 3: request with correct logical plan --> expect to have fragment metadata + scheduler.cleanupMetricsForInactiveUser("test2") + + lp := createTestLogicalPlan(t, time.Now(), time.Now(), 0, "up") + bytesLp, err := logicalplan.Marshal(lp.Root()) + form := url.Values{} + form.Set("plan", string(bytesLp)) // this is to imitate how the real format of http request body + require.NoError(t, err) + frontendToScheduler(t, frontendLoop, &schedulerpb.FrontendToScheduler{ + Type: schedulerpb.ENQUEUE, + QueryID: 3, + UserID: "test3", + HttpRequest: &httpgrpc.HTTPRequest{Method: "POST", Url: "/hello", Body: []byte(form.Encode())}, + }) + require.NoError(t, promtest.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_query_scheduler_queue_length Number of queries in the queue. + # TYPE cortex_query_scheduler_queue_length gauge + cortex_query_scheduler_queue_length{priority="0",type="fifo",user="test3"} 1 + # HELP cortex_request_queue_requests_total Total number of query requests going to the request queue. + # TYPE cortex_request_queue_requests_total counter + cortex_request_queue_requests_total{priority="0",user="test3"} 1 + `), "cortex_query_scheduler_queue_length", "cortex_request_queue_requests_total")) + + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{QuerierID: "querier-1", QuerierAddress: "localhost:8000"})) + + s3, err := querierLoop.Recv() + require.NoError(t, err) + require.NotEmpty(t, s3.FragmentID) + require.Equal(t, uint64(3), s3.QueryID) + require.Empty(t, s3.ChildFragmentID) // there is only one fragment for the logical plan, so no child plan_fragments + require.Empty(t, s3.ChildAddr) + require.Equal(t, s3.HttpRequest.Body, []byte(form.Encode())) + require.True(t, s3.IsRoot) +} + +// TestQuerierLoopClient_WithLogicalPlan_Fragmented checks if fragments of the logical plan +// can be picked up successfully and have the correct metadata with them +// It also tests scheduler coordination hashmap. +// It acts as an integration test for the scheduler for distributed query execution +// (this test relates to the design of distributed optimizer + fragmenter, so it needs to be adjusted accordingly) + +func TestQuerierLoopClient_WithLogicalPlan_Fragmented(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + + scheduler, frontendClient, querierClient := setupScheduler(t, reg) + scheduler.distributedExecEnabled = true + + frontendLoop := initFrontendLoop(t, frontendClient, "frontend-12345") + querierLoop, err := querierClient.QuerierLoop(context.Background()) + require.NoError(t, err) + + lp_long := createTestLogicalPlan(t, time.Now(), time.Now(), 0, "sum(rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])) + sum(rate(node_memory_Active_bytes[5m]))") + bytesLp_long, err := logicalplan.Marshal(lp_long.Root()) + form_long := url.Values{} + form_long.Set("plan", string(bytesLp_long)) // this is to imitate how the real format of http request body + require.NoError(t, err) + frontendToScheduler(t, frontendLoop, &schedulerpb.FrontendToScheduler{ + Type: schedulerpb.ENQUEUE, + QueryID: 4, + UserID: "test", + HttpRequest: &httpgrpc.HTTPRequest{Method: "POST", Url: "/hello", Body: []byte(form_long.Encode())}, + }) + require.NoError(t, promtest.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_query_scheduler_queue_length Number of queries in the queue. + # TYPE cortex_query_scheduler_queue_length gauge + cortex_query_scheduler_queue_length{priority="0",type="fifo",user="test"} 3 + # HELP cortex_request_queue_requests_total Total number of query requests going to the request queue. + # TYPE cortex_request_queue_requests_total counter + cortex_request_queue_requests_total{priority="0",user="test"} 3 + `), "cortex_query_scheduler_queue_length", "cortex_request_queue_requests_total")) + + requestLeft := getNumOfPendingRequestsLeft(scheduler) + require.Equal(t, 3, requestLeft) + + // fragment 1 + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{QuerierID: "querier-1", QuerierAddress: "localhost:8000"})) + s1, err := querierLoop.Recv() + require.NoError(t, err) + require.NotEmpty(t, s1.FragmentID) + require.Equal(t, uint64(4), s1.QueryID) + require.Empty(t, s1.ChildFragmentID) // there is only one fragment for the logical plan, so no child plan_fragments + require.Empty(t, s1.ChildAddr) + require.False(t, s1.IsRoot) + + // check if the new address is added to the scheduler's table + addr1, exist := scheduler.fragmentTable.GetMapping(s1.QueryID, []uint64{s1.FragmentID}) + require.True(t, exist) + require.Equal(t, addr1[0], "localhost:8000") + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{})) // mark ready for the next task + + // fragment 2 + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{QuerierID: "querier-1", QuerierAddress: "localhost:8000"})) + s2, err := querierLoop.Recv() + require.NoError(t, err) + require.NotEmpty(t, s2.FragmentID) + require.Equal(t, uint64(4), s2.QueryID) + require.Empty(t, s2.ChildFragmentID) // there is only one fragment for the logical plan, so no child plan_fragments + require.Empty(t, s2.ChildAddr) + require.False(t, s2.IsRoot) + + addr2, exist := scheduler.fragmentTable.GetMapping(s2.QueryID, []uint64{s2.FragmentID}) + require.True(t, exist) + require.Equal(t, addr2[0], "localhost:8000") + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{})) // mark ready for the next task + + // fragment 3 + require.NoError(t, querierLoop.Send(&schedulerpb.QuerierToScheduler{QuerierID: "querier-1", QuerierAddress: "localhost:8000"})) + s3, err := querierLoop.Recv() + require.NoError(t, err) + require.NotEmpty(t, s3.FragmentID) + require.Equal(t, uint64(4), s3.QueryID) + require.Equal(t, s3.ChildFragmentID, []uint64{s1.FragmentID, s2.FragmentID}) // equal to the child fragment IDs + require.NotEmpty(t, s3.ChildAddr, []string{addr1[0], addr2[0]}) + require.True(t, s3.IsRoot) + + // Because this fragment is the root of the query, so the hashmap will clear all related fragments out + addr, exist := scheduler.fragmentTable.GetMapping(s3.QueryID, []uint64{s3.FragmentID}) + require.False(t, exist) + require.Empty(t, addr) + + requestLeft = getNumOfPendingRequestsLeft(scheduler) + require.Equal(t, 0, requestLeft) +} + +func createTestLogicalPlan(t *testing.T, startTime time.Time, endTime time.Time, step time.Duration, q string) logicalplan.Plan { + qOpts := query.Options{ + Start: startTime, + End: startTime, + Step: 0, + StepsBatch: 10, + LookbackDelta: 0, + EnablePerStepStats: false, + } + + if step != 0 { + qOpts.End = endTime + qOpts.Step = step + } + + expr, err := parser.NewParser(q, parser.WithFunctions(parser.Functions)).ParseExpr() + require.NoError(t, err) + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: false, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) + dOptimizer := tripperware.DistributedOptimizer{} + dOptimizedPlanNode, _ := dOptimizer.Optimize(optimizedPlan.Root(), &qOpts) + lp := logicalplan.New(dOptimizedPlanNode, &qOpts, planOpts) + + return lp +} + func initFrontendLoop(t *testing.T, client schedulerpb.SchedulerForFrontendClient, frontendAddr string) schedulerpb.SchedulerForFrontend_FrontendLoopClient { loop, err := client.FrontendLoop(context.Background()) require.NoError(t, err) @@ -501,6 +710,12 @@ func verifyNoPendingRequestsLeft(t *testing.T, scheduler *Scheduler) { }) } +func getNumOfPendingRequestsLeft(scheduler *Scheduler) int { + scheduler.pendingRequestsMu.Lock() + defer scheduler.pendingRequestsMu.Unlock() + return len(scheduler.pendingRequests) +} + type frontendMock struct { mu sync.Mutex resp map[uint64]*httpgrpc.HTTPResponse diff --git a/pkg/scheduler/schedulerpb/scheduler.pb.go b/pkg/scheduler/schedulerpb/scheduler.pb.go index d3288f95b39..875c8fda13b 100644 --- a/pkg/scheduler/schedulerpb/scheduler.pb.go +++ b/pkg/scheduler/schedulerpb/scheduler.pb.go @@ -85,7 +85,8 @@ func (SchedulerToFrontendStatus) EnumDescriptor() ([]byte, []int) { // Querier reports its own clientID when it connects, so that scheduler knows how many *different* queriers are connected. // To signal that querier is ready to accept another request, querier sends empty message. type QuerierToScheduler struct { - QuerierID string `protobuf:"bytes,1,opt,name=querierID,proto3" json:"querierID,omitempty"` + QuerierID string `protobuf:"bytes,1,opt,name=querierID,proto3" json:"querierID,omitempty"` + QuerierAddress string `protobuf:"bytes,2,opt,name=querierAddress,proto3" json:"querierAddress,omitempty"` } func (m *QuerierToScheduler) Reset() { *m = QuerierToScheduler{} } @@ -127,6 +128,13 @@ func (m *QuerierToScheduler) GetQuerierID() string { return "" } +func (m *QuerierToScheduler) GetQuerierAddress() string { + if m != nil { + return m.QuerierAddress + } + return "" +} + type SchedulerToQuerier struct { // Query ID as reported by frontend. When querier sends the response back to frontend (using frontendAddress), // it identifies the query by using this ID. @@ -139,6 +147,14 @@ type SchedulerToQuerier struct { // Whether query statistics tracking should be enabled. The response will include // statistics only when this option is enabled. StatsEnabled bool `protobuf:"varint,5,opt,name=statsEnabled,proto3" json:"statsEnabled,omitempty"` + // The ID of logical query plan fragment. + FragmentID uint64 `protobuf:"varint,6,opt,name=fragmentID,proto3" json:"fragmentID,omitempty"` + // The IDs of the child fragments + ChildFragmentID []uint64 `protobuf:"varint,7,rep,packed,name=childFragmentID,proto3" json:"childFragmentID,omitempty"` + // The address of the queriers that have the child fragments + ChildAddr []string `protobuf:"bytes,8,rep,name=childAddr,proto3" json:"childAddr,omitempty"` + // Whether the current fragment is the root + IsRoot bool `protobuf:"varint,9,opt,name=isRoot,proto3" json:"isRoot,omitempty"` } func (m *SchedulerToQuerier) Reset() { *m = SchedulerToQuerier{} } @@ -208,6 +224,34 @@ func (m *SchedulerToQuerier) GetStatsEnabled() bool { return false } +func (m *SchedulerToQuerier) GetFragmentID() uint64 { + if m != nil { + return m.FragmentID + } + return 0 +} + +func (m *SchedulerToQuerier) GetChildFragmentID() []uint64 { + if m != nil { + return m.ChildFragmentID + } + return nil +} + +func (m *SchedulerToQuerier) GetChildAddr() []string { + if m != nil { + return m.ChildAddr + } + return nil +} + +func (m *SchedulerToQuerier) GetIsRoot() bool { + if m != nil { + return m.IsRoot + } + return false +} + type FrontendToScheduler struct { Type FrontendToSchedulerType `protobuf:"varint,1,opt,name=type,proto3,enum=schedulerpb.FrontendToSchedulerType" json:"type,omitempty"` // Used by INIT message. Will be put into all requests passed to querier. @@ -438,48 +482,52 @@ func init() { func init() { proto.RegisterFile("scheduler.proto", fileDescriptor_2b3fc28395a6d9c5) } var fileDescriptor_2b3fc28395a6d9c5 = []byte{ - // 644 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0x4f, 0x4f, 0xdb, 0x4e, - 0x10, 0xf5, 0x86, 0x24, 0xc0, 0x84, 0xdf, 0x0f, 0x77, 0x81, 0x36, 0x8d, 0xe8, 0x12, 0x45, 0x55, - 0x95, 0x72, 0x48, 0xaa, 0xb4, 0x52, 0x7b, 0x40, 0x95, 0x52, 0x30, 0x25, 0x2a, 0x75, 0x60, 0xb3, - 0x51, 0xff, 0x5c, 0x22, 0x92, 0x2c, 0x09, 0x02, 0xbc, 0x66, 0x6d, 0x17, 0xe5, 0xd6, 0x63, 0x8f, - 0xfd, 0x18, 0xfd, 0x28, 0xbd, 0x54, 0xe2, 0xc8, 0xa1, 0x87, 0x62, 0x2e, 0x3d, 0xf2, 0x11, 0xaa, - 0x38, 0x76, 0xea, 0xa4, 0x0e, 0x70, 0x9b, 0x1d, 0xbf, 0xe7, 0x9d, 0xf7, 0x66, 0x66, 0x61, 0xde, - 0x6a, 0x75, 0x79, 0xdb, 0x39, 0xe2, 0xb2, 0x60, 0x4a, 0x61, 0x0b, 0x9c, 0x1a, 0x26, 0xcc, 0x66, - 0x66, 0xb1, 0x23, 0x3a, 0xc2, 0xcb, 0x17, 0xfb, 0xd1, 0x00, 0x92, 0x79, 0xd6, 0x39, 0xb0, 0xbb, - 0x4e, 0xb3, 0xd0, 0x12, 0xc7, 0xc5, 0x53, 0xbe, 0xf7, 0x89, 0x9f, 0x0a, 0x79, 0x68, 0x15, 0x5b, - 0xe2, 0xf8, 0x58, 0x18, 0xc5, 0xae, 0x6d, 0x9b, 0x1d, 0x69, 0xb6, 0x86, 0xc1, 0x80, 0x95, 0x2b, - 0x01, 0xde, 0x75, 0xb8, 0x3c, 0xe0, 0x92, 0x89, 0x5a, 0x70, 0x07, 0x5e, 0x86, 0xd9, 0x93, 0x41, - 0xb6, 0xb2, 0x91, 0x46, 0x59, 0x94, 0x9f, 0xa5, 0x7f, 0x13, 0xb9, 0x1f, 0x08, 0xf0, 0x10, 0xcb, - 0x84, 0xcf, 0xc7, 0x69, 0x98, 0xee, 0x63, 0x7a, 0x3e, 0x25, 0x4e, 0x83, 0x23, 0x7e, 0x0e, 0xa9, - 0xfe, 0xb5, 0x94, 0x9f, 0x38, 0xdc, 0xb2, 0xd3, 0xb1, 0x2c, 0xca, 0xa7, 0x4a, 0x4b, 0x85, 0x61, - 0x29, 0x5b, 0x8c, 0xed, 0xf8, 0x1f, 0x69, 0x18, 0x89, 0xf3, 0x30, 0xbf, 0x2f, 0x85, 0x61, 0x73, - 0xa3, 0x5d, 0x6e, 0xb7, 0x25, 0xb7, 0xac, 0xf4, 0x94, 0x57, 0xcd, 0x78, 0x1a, 0xdf, 0x85, 0xa4, - 0x63, 0x79, 0xe5, 0xc6, 0x3d, 0x80, 0x7f, 0xc2, 0x39, 0x98, 0xb3, 0xec, 0x3d, 0xdb, 0xd2, 0x8c, - 0xbd, 0xe6, 0x11, 0x6f, 0xa7, 0x13, 0x59, 0x94, 0x9f, 0xa1, 0x23, 0xb9, 0xdc, 0x97, 0x18, 0x2c, - 0x6c, 0xfa, 0xff, 0x0b, 0xbb, 0xf0, 0x02, 0xe2, 0x76, 0xcf, 0xe4, 0x9e, 0x9a, 0xff, 0x4b, 0x0f, - 0x0b, 0xa1, 0x1e, 0x14, 0x22, 0xf0, 0xac, 0x67, 0x72, 0xea, 0x31, 0xa2, 0xea, 0x8e, 0x45, 0xd7, - 0x1d, 0x32, 0x6d, 0x6a, 0xd4, 0xb4, 0x49, 0x8a, 0xc6, 0xcc, 0x4c, 0xdc, 0xda, 0xcc, 0x71, 0x2b, - 0x92, 0x11, 0x56, 0x1c, 0xc2, 0x42, 0xa8, 0xb3, 0x81, 0x48, 0xfc, 0x12, 0x92, 0x7d, 0x98, 0x63, - 0xf9, 0x5e, 0x3c, 0x1a, 0xf1, 0x22, 0x82, 0x51, 0xf3, 0xd0, 0xd4, 0x67, 0xe1, 0x45, 0x48, 0x70, - 0x29, 0x85, 0xf4, 0x5d, 0x18, 0x1c, 0x72, 0x6b, 0xb0, 0xac, 0x0b, 0xfb, 0x60, 0xbf, 0xe7, 0x4f, - 0x50, 0xad, 0xeb, 0xd8, 0x6d, 0x71, 0x6a, 0x04, 0x05, 0x5f, 0x3f, 0x85, 0x2b, 0xf0, 0x60, 0x02, - 0xdb, 0x32, 0x85, 0x61, 0xf1, 0xd5, 0x35, 0xb8, 0x37, 0xa1, 0x4b, 0x78, 0x06, 0xe2, 0x15, 0xbd, - 0xc2, 0x54, 0x05, 0xa7, 0x60, 0x5a, 0xd3, 0x77, 0xeb, 0x5a, 0x5d, 0x53, 0x11, 0x06, 0x48, 0xae, - 0x97, 0xf5, 0x75, 0x6d, 0x5b, 0x8d, 0xad, 0xb6, 0xe0, 0xfe, 0x44, 0x5d, 0x38, 0x09, 0xb1, 0xea, - 0x1b, 0x55, 0xc1, 0x59, 0x58, 0x66, 0xd5, 0x6a, 0xe3, 0x6d, 0x59, 0xff, 0xd0, 0xa0, 0xda, 0x6e, - 0x5d, 0xab, 0xb1, 0x5a, 0x63, 0x47, 0xa3, 0x0d, 0xa6, 0xe9, 0x65, 0x9d, 0xa9, 0x08, 0xcf, 0x42, - 0x42, 0xa3, 0xb4, 0x4a, 0xd5, 0x18, 0xbe, 0x03, 0xff, 0xd5, 0xb6, 0xea, 0x8c, 0x55, 0xf4, 0xd7, - 0x8d, 0x8d, 0xea, 0x3b, 0x5d, 0x9d, 0x2a, 0xfd, 0x44, 0x21, 0xbf, 0x37, 0x85, 0x0c, 0x56, 0xa9, - 0x0e, 0x29, 0x3f, 0xdc, 0x16, 0xc2, 0xc4, 0x2b, 0x23, 0x76, 0xff, 0xbb, 0xaf, 0x99, 0x95, 0x49, - 0xfd, 0xf0, 0xb1, 0x39, 0x25, 0x8f, 0x9e, 0x20, 0x6c, 0xc0, 0x52, 0xa4, 0x65, 0xf8, 0xf1, 0x08, - 0xff, 0xba, 0xa6, 0x64, 0x56, 0x6f, 0x03, 0x1d, 0x74, 0xa0, 0x64, 0xc2, 0x62, 0x58, 0xdd, 0x70, - 0x9c, 0xde, 0xc3, 0x5c, 0x10, 0x7b, 0xfa, 0xb2, 0x37, 0xad, 0x56, 0x26, 0x7b, 0xd3, 0xc0, 0x0d, - 0x14, 0xbe, 0x2a, 0x9f, 0x5d, 0x10, 0xe5, 0xfc, 0x82, 0x28, 0x57, 0x17, 0x04, 0x7d, 0x76, 0x09, - 0xfa, 0xe6, 0x12, 0xf4, 0xdd, 0x25, 0xe8, 0xcc, 0x25, 0xe8, 0x97, 0x4b, 0xd0, 0x6f, 0x97, 0x28, - 0x57, 0x2e, 0x41, 0x5f, 0x2f, 0x89, 0x72, 0x76, 0x49, 0x94, 0xf3, 0x4b, 0xa2, 0x7c, 0x0c, 0xbf, - 0xae, 0xcd, 0xa4, 0xf7, 0x30, 0x3e, 0xfd, 0x13, 0x00, 0x00, 0xff, 0xff, 0x88, 0x0c, 0xfe, 0x56, - 0x84, 0x05, 0x00, 0x00, + // 713 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4d, 0x4f, 0x1a, 0x51, + 0x14, 0x9d, 0xc7, 0x97, 0x72, 0xb1, 0x4a, 0x9f, 0xda, 0x52, 0x62, 0x9f, 0x64, 0xd2, 0x18, 0xea, + 0x02, 0x1a, 0xda, 0xa4, 0x5d, 0x98, 0x26, 0x54, 0xc7, 0x4a, 0x6a, 0x07, 0x7d, 0x0c, 0x69, 0xeb, + 0x86, 0xf0, 0xf1, 0x04, 0xa2, 0xcc, 0x1b, 0x67, 0x86, 0x1a, 0x76, 0x5d, 0x76, 0xd9, 0x9f, 0xd1, + 0x9f, 0xd2, 0x4d, 0x13, 0x97, 0x2e, 0xba, 0xa8, 0xb8, 0xe9, 0xd2, 0x9f, 0xd0, 0xcc, 0x63, 0x06, + 0x07, 0x04, 0x75, 0xf7, 0xee, 0xe1, 0xdc, 0x99, 0x73, 0xcf, 0xb9, 0x6f, 0x80, 0x05, 0xab, 0xde, + 0x62, 0x8d, 0xee, 0x31, 0x33, 0x33, 0x86, 0xc9, 0x6d, 0x8e, 0x63, 0x43, 0xc0, 0xa8, 0x25, 0x97, + 0x9a, 0xbc, 0xc9, 0x05, 0x9e, 0x75, 0x4e, 0x03, 0x4a, 0xf2, 0x55, 0xb3, 0x6d, 0xb7, 0xba, 0xb5, + 0x4c, 0x9d, 0x77, 0xb2, 0xa7, 0xac, 0xfa, 0x95, 0x9d, 0x72, 0xf3, 0xc8, 0xca, 0xd6, 0x79, 0xa7, + 0xc3, 0xf5, 0x6c, 0xcb, 0xb6, 0x8d, 0xa6, 0x69, 0xd4, 0x87, 0x87, 0x41, 0x97, 0x7c, 0x00, 0x78, + 0xbf, 0xcb, 0xcc, 0x36, 0x33, 0x35, 0x5e, 0xf2, 0xde, 0x81, 0x57, 0x20, 0x7a, 0x32, 0x40, 0x0b, + 0x5b, 0x09, 0x94, 0x42, 0xe9, 0x28, 0xbd, 0x06, 0xf0, 0x1a, 0xcc, 0xbb, 0x45, 0xbe, 0xd1, 0x30, + 0x99, 0x65, 0x25, 0x02, 0x82, 0x32, 0x86, 0xca, 0xbf, 0x03, 0x80, 0x87, 0xcf, 0xd4, 0xb8, 0xfb, + 0x1e, 0x9c, 0x80, 0x19, 0x87, 0xd8, 0x73, 0x1f, 0x1d, 0xa2, 0x5e, 0x89, 0x5f, 0x43, 0xcc, 0x91, + 0x47, 0xd9, 0x49, 0x97, 0x59, 0xb6, 0x78, 0x6a, 0x2c, 0xb7, 0x9c, 0x19, 0x4a, 0xde, 0xd1, 0xb4, + 0x3d, 0xf7, 0x47, 0xea, 0x67, 0xe2, 0x34, 0x2c, 0x1c, 0x9a, 0x5c, 0xb7, 0x99, 0xde, 0xf0, 0x24, + 0x05, 0x85, 0xa4, 0x71, 0x18, 0x3f, 0x82, 0x48, 0xd7, 0x12, 0x63, 0x85, 0x04, 0xc1, 0xad, 0xb0, + 0x0c, 0x73, 0x96, 0x5d, 0xb5, 0x2d, 0x45, 0xaf, 0xd6, 0x8e, 0x59, 0x23, 0x11, 0x4e, 0xa1, 0xf4, + 0x2c, 0x1d, 0xc1, 0x30, 0x01, 0x38, 0x34, 0xab, 0xcd, 0x0e, 0xd3, 0xed, 0xc2, 0x56, 0x22, 0x22, + 0xb4, 0xfb, 0x10, 0x47, 0x45, 0xbd, 0xd5, 0x3e, 0x6e, 0x6c, 0x5f, 0x93, 0x66, 0x52, 0xc1, 0x74, + 0x88, 0x8e, 0xc3, 0x8e, 0xbf, 0x02, 0x72, 0x54, 0x25, 0x66, 0x53, 0x41, 0xc7, 0xdf, 0x21, 0xe0, + 0x68, 0x6c, 0x5b, 0x94, 0x73, 0x3b, 0x11, 0x15, 0x2a, 0xdc, 0x4a, 0xfe, 0x1e, 0x80, 0xc5, 0x6d, + 0x77, 0x1e, 0x7f, 0x5a, 0x6f, 0x20, 0x64, 0xf7, 0x0c, 0x26, 0xdc, 0x9c, 0xcf, 0x3d, 0xcb, 0xf8, + 0x76, 0x25, 0x33, 0x81, 0xaf, 0xf5, 0x0c, 0x46, 0x45, 0xc7, 0x24, 0xdf, 0x02, 0x93, 0x7d, 0xf3, + 0x85, 0x16, 0x1c, 0x0d, 0x6d, 0x9a, 0xa3, 0x63, 0x61, 0x86, 0xef, 0x1d, 0xe6, 0x78, 0x14, 0x91, + 0x9b, 0x51, 0xc8, 0x47, 0xb0, 0xe8, 0xdb, 0x2c, 0x6f, 0x48, 0xfc, 0x16, 0x22, 0x0e, 0xad, 0x6b, + 0xb9, 0x5e, 0xac, 0x8d, 0x78, 0x31, 0xa1, 0xa3, 0x24, 0xd8, 0xd4, 0xed, 0xc2, 0x4b, 0x10, 0x66, + 0xa6, 0xc9, 0x4d, 0xd7, 0x85, 0x41, 0x21, 0x6f, 0xc0, 0x8a, 0xca, 0xed, 0xf6, 0x61, 0xcf, 0xdd, + 0xe0, 0x52, 0xab, 0x6b, 0x37, 0xf8, 0xa9, 0xee, 0x09, 0xbe, 0xf5, 0xb6, 0xc8, 0xab, 0xf0, 0x74, + 0x4a, 0xb7, 0x65, 0x70, 0xdd, 0x62, 0xeb, 0x1b, 0xf0, 0x78, 0x4a, 0x4a, 0x78, 0x16, 0x42, 0x05, + 0xb5, 0xa0, 0xc5, 0x25, 0x1c, 0x83, 0x19, 0x45, 0xdd, 0x2f, 0x2b, 0x65, 0x25, 0x8e, 0x30, 0x40, + 0x64, 0x33, 0xaf, 0x6e, 0x2a, 0xbb, 0xf1, 0xc0, 0x7a, 0x1d, 0x9e, 0x4c, 0x9d, 0x0b, 0x47, 0x20, + 0x50, 0xfc, 0x10, 0x97, 0x70, 0x0a, 0x56, 0xb4, 0x62, 0xb1, 0xf2, 0x31, 0xaf, 0x7e, 0xa9, 0x50, + 0x65, 0xbf, 0xac, 0x94, 0xb4, 0x52, 0x65, 0x4f, 0xa1, 0x15, 0x4d, 0x51, 0xf3, 0xaa, 0x16, 0x47, + 0x38, 0x0a, 0x61, 0x85, 0xd2, 0x22, 0x8d, 0x07, 0xf0, 0x43, 0x78, 0x50, 0xda, 0x29, 0x6b, 0x5a, + 0x41, 0x7d, 0x5f, 0xd9, 0x2a, 0x7e, 0x52, 0xe3, 0xc1, 0xdc, 0x1f, 0xe4, 0xf3, 0x7b, 0x9b, 0x9b, + 0xde, 0x55, 0x2e, 0x43, 0xcc, 0x3d, 0xee, 0x72, 0x6e, 0xe0, 0xd5, 0x11, 0xbb, 0x6f, 0x7e, 0x57, + 0x92, 0xab, 0xd3, 0xf2, 0x70, 0xb9, 0xb2, 0x94, 0x46, 0x2f, 0x10, 0xd6, 0x61, 0x79, 0xa2, 0x65, + 0xf8, 0xf9, 0x48, 0xff, 0x6d, 0xa1, 0x24, 0xd7, 0xef, 0x43, 0x1d, 0x24, 0x90, 0x33, 0x60, 0xc9, + 0x3f, 0xdd, 0x70, 0x9d, 0x3e, 0xc3, 0x9c, 0x77, 0x16, 0xf3, 0xa5, 0xee, 0xba, 0x5a, 0xc9, 0xd4, + 0x5d, 0x0b, 0x37, 0x98, 0xf0, 0x5d, 0xfe, 0xec, 0x82, 0x48, 0xe7, 0x17, 0x44, 0xba, 0xba, 0x20, + 0xe8, 0x5b, 0x9f, 0xa0, 0x9f, 0x7d, 0x82, 0x7e, 0xf5, 0x09, 0x3a, 0xeb, 0x13, 0xf4, 0xb7, 0x4f, + 0xd0, 0xbf, 0x3e, 0x91, 0xae, 0xfa, 0x04, 0xfd, 0xb8, 0x24, 0xd2, 0xd9, 0x25, 0x91, 0xce, 0x2f, + 0x89, 0x74, 0xe0, 0xff, 0x17, 0xa8, 0x45, 0xc4, 0x07, 0xfc, 0xe5, 0xff, 0x00, 0x00, 0x00, 0xff, + 0xff, 0x8d, 0x51, 0x86, 0x0b, 0x2c, 0x06, 0x00, 0x00, } func (x FrontendToSchedulerType) String() string { @@ -518,6 +566,9 @@ func (this *QuerierToScheduler) Equal(that interface{}) bool { if this.QuerierID != that1.QuerierID { return false } + if this.QuerierAddress != that1.QuerierAddress { + return false + } return true } func (this *SchedulerToQuerier) Equal(that interface{}) bool { @@ -554,6 +605,28 @@ func (this *SchedulerToQuerier) Equal(that interface{}) bool { if this.StatsEnabled != that1.StatsEnabled { return false } + if this.FragmentID != that1.FragmentID { + return false + } + if len(this.ChildFragmentID) != len(that1.ChildFragmentID) { + return false + } + for i := range this.ChildFragmentID { + if this.ChildFragmentID[i] != that1.ChildFragmentID[i] { + return false + } + } + if len(this.ChildAddr) != len(that1.ChildAddr) { + return false + } + for i := range this.ChildAddr { + if this.ChildAddr[i] != that1.ChildAddr[i] { + return false + } + } + if this.IsRoot != that1.IsRoot { + return false + } return true } func (this *FrontendToScheduler) Equal(that interface{}) bool { @@ -671,9 +744,10 @@ func (this *QuerierToScheduler) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 5) + s := make([]string, 0, 6) s = append(s, "&schedulerpb.QuerierToScheduler{") s = append(s, "QuerierID: "+fmt.Sprintf("%#v", this.QuerierID)+",\n") + s = append(s, "QuerierAddress: "+fmt.Sprintf("%#v", this.QuerierAddress)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -681,7 +755,7 @@ func (this *SchedulerToQuerier) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 13) s = append(s, "&schedulerpb.SchedulerToQuerier{") s = append(s, "QueryID: "+fmt.Sprintf("%#v", this.QueryID)+",\n") if this.HttpRequest != nil { @@ -690,6 +764,10 @@ func (this *SchedulerToQuerier) GoString() string { s = append(s, "FrontendAddress: "+fmt.Sprintf("%#v", this.FrontendAddress)+",\n") s = append(s, "UserID: "+fmt.Sprintf("%#v", this.UserID)+",\n") s = append(s, "StatsEnabled: "+fmt.Sprintf("%#v", this.StatsEnabled)+",\n") + s = append(s, "FragmentID: "+fmt.Sprintf("%#v", this.FragmentID)+",\n") + s = append(s, "ChildFragmentID: "+fmt.Sprintf("%#v", this.ChildFragmentID)+",\n") + s = append(s, "ChildAddr: "+fmt.Sprintf("%#v", this.ChildAddr)+",\n") + s = append(s, "IsRoot: "+fmt.Sprintf("%#v", this.IsRoot)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -1048,6 +1126,13 @@ func (m *QuerierToScheduler) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.QuerierAddress) > 0 { + i -= len(m.QuerierAddress) + copy(dAtA[i:], m.QuerierAddress) + i = encodeVarintScheduler(dAtA, i, uint64(len(m.QuerierAddress))) + i-- + dAtA[i] = 0x12 + } if len(m.QuerierID) > 0 { i -= len(m.QuerierID) copy(dAtA[i:], m.QuerierID) @@ -1078,6 +1163,48 @@ func (m *SchedulerToQuerier) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.IsRoot { + i-- + if m.IsRoot { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x48 + } + if len(m.ChildAddr) > 0 { + for iNdEx := len(m.ChildAddr) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ChildAddr[iNdEx]) + copy(dAtA[i:], m.ChildAddr[iNdEx]) + i = encodeVarintScheduler(dAtA, i, uint64(len(m.ChildAddr[iNdEx]))) + i-- + dAtA[i] = 0x42 + } + } + if len(m.ChildFragmentID) > 0 { + dAtA2 := make([]byte, len(m.ChildFragmentID)*10) + var j1 int + for _, num := range m.ChildFragmentID { + for num >= 1<<7 { + dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA2[j1] = uint8(num) + j1++ + } + i -= j1 + copy(dAtA[i:], dAtA2[:j1]) + i = encodeVarintScheduler(dAtA, i, uint64(j1)) + i-- + dAtA[i] = 0x3a + } + if m.FragmentID != 0 { + i = encodeVarintScheduler(dAtA, i, uint64(m.FragmentID)) + i-- + dAtA[i] = 0x30 + } if m.StatsEnabled { i-- if m.StatsEnabled { @@ -1300,6 +1427,10 @@ func (m *QuerierToScheduler) Size() (n int) { if l > 0 { n += 1 + l + sovScheduler(uint64(l)) } + l = len(m.QuerierAddress) + if l > 0 { + n += 1 + l + sovScheduler(uint64(l)) + } return n } @@ -1327,6 +1458,25 @@ func (m *SchedulerToQuerier) Size() (n int) { if m.StatsEnabled { n += 2 } + if m.FragmentID != 0 { + n += 1 + sovScheduler(uint64(m.FragmentID)) + } + if len(m.ChildFragmentID) > 0 { + l = 0 + for _, e := range m.ChildFragmentID { + l += sovScheduler(uint64(e)) + } + n += 1 + sovScheduler(uint64(l)) + l + } + if len(m.ChildAddr) > 0 { + for _, s := range m.ChildAddr { + l = len(s) + n += 1 + l + sovScheduler(uint64(l)) + } + } + if m.IsRoot { + n += 2 + } return n } @@ -1410,6 +1560,7 @@ func (this *QuerierToScheduler) String() string { } s := strings.Join([]string{`&QuerierToScheduler{`, `QuerierID:` + fmt.Sprintf("%v", this.QuerierID) + `,`, + `QuerierAddress:` + fmt.Sprintf("%v", this.QuerierAddress) + `,`, `}`, }, "") return s @@ -1424,6 +1575,10 @@ func (this *SchedulerToQuerier) String() string { `FrontendAddress:` + fmt.Sprintf("%v", this.FrontendAddress) + `,`, `UserID:` + fmt.Sprintf("%v", this.UserID) + `,`, `StatsEnabled:` + fmt.Sprintf("%v", this.StatsEnabled) + `,`, + `FragmentID:` + fmt.Sprintf("%v", this.FragmentID) + `,`, + `ChildFragmentID:` + fmt.Sprintf("%v", this.ChildFragmentID) + `,`, + `ChildAddr:` + fmt.Sprintf("%v", this.ChildAddr) + `,`, + `IsRoot:` + fmt.Sprintf("%v", this.IsRoot) + `,`, `}`, }, "") return s @@ -1542,6 +1697,38 @@ func (m *QuerierToScheduler) Unmarshal(dAtA []byte) error { } m.QuerierID = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QuerierAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + 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 ErrInvalidLengthScheduler + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScheduler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.QuerierAddress = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScheduler(dAtA[iNdEx:]) @@ -1734,6 +1921,153 @@ func (m *SchedulerToQuerier) Unmarshal(dAtA []byte) error { } } m.StatsEnabled = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FragmentID", wireType) + } + m.FragmentID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FragmentID |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ChildFragmentID = append(m.ChildFragmentID, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScheduler + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScheduler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ChildFragmentID) == 0 { + m.ChildFragmentID = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ChildFragmentID = append(m.ChildFragmentID, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ChildFragmentID", wireType) + } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChildAddr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + 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 ErrInvalidLengthScheduler + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScheduler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ChildAddr = append(m.ChildAddr, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsRoot", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScheduler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsRoot = bool(v != 0) default: iNdEx = preIndex skippy, err := skipScheduler(dAtA[iNdEx:]) diff --git a/pkg/scheduler/schedulerpb/scheduler.proto b/pkg/scheduler/schedulerpb/scheduler.proto index eea28717b83..f7c2d3a97f8 100644 --- a/pkg/scheduler/schedulerpb/scheduler.proto +++ b/pkg/scheduler/schedulerpb/scheduler.proto @@ -27,7 +27,10 @@ service SchedulerForQuerier { // Querier reports its own clientID when it connects, so that scheduler knows how many *different* queriers are connected. // To signal that querier is ready to accept another request, querier sends empty message. message QuerierToScheduler { + string querierID = 1; + + string querierAddress = 2; } message SchedulerToQuerier { @@ -45,6 +48,20 @@ message SchedulerToQuerier { // Whether query statistics tracking should be enabled. The response will include // statistics only when this option is enabled. bool statsEnabled = 5; + + // Below are the meta data that will be used for distributed execution + + // The ID of logical query plan fragment. + uint64 fragmentID = 6; + + // The IDs of the child fragments + repeated uint64 childFragmentID = 7; + + // The address of the queriers that have the child fragments + repeated string childAddr = 8; + + // Whether the current fragment is the root + bool isRoot = 9; } // Scheduler interface exposed to Frontend. Frontend can enqueue and cancel requests.