diff --git a/api/config.go b/api/config.go index f55b0dfb39..48c13ae660 100644 --- a/api/config.go +++ b/api/config.go @@ -8,6 +8,7 @@ import ( "time" "github.com/grafana/globalconf" + "github.com/grafana/metrictank/expr" log "github.com/sirupsen/logrus" ) @@ -28,6 +29,7 @@ var ( getTargetsConcurrency int tagdbDefaultLimit uint speculationThreshold float64 + optimizations expr.Optimizations graphiteProxy *httputil.ReverseProxy timeZone *time.Location @@ -49,6 +51,8 @@ func ConfigSetup() { apiCfg.IntVar(&getTargetsConcurrency, "get-targets-concurrency", 20, "maximum number of concurrent threads for fetching data on the local node. Each thread handles a single series.") apiCfg.UintVar(&tagdbDefaultLimit, "tagdb-default-limit", 100, "default limit for tagdb query results, can be overridden with query parameter \"limit\"") apiCfg.Float64Var(&speculationThreshold, "speculation-threshold", 1, "ratio of peer responses after which speculation is used. Set to 1 to disable.") + apiCfg.BoolVar(&optimizations.PreNormalization, "pre-normalization", true, "enable pre-normalization optimization") + apiCfg.BoolVar(&optimizations.MDP, "mdp-optimization", false, "enable MaxDataPoints optimization (experimental)") globalconf.Register("http", apiCfg, flag.ExitOnError) } diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 06bffbbf91..0c89f71a9b 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -329,7 +329,7 @@ LOOP: } -// getTarget returns the series for the request in canonical form. +// getTarget returns the series for the request in canonical form with respect to their OutInterval // as ConsolidateContext just processes what it's been given (not "stable" or bucket-aligned to the output interval) // we simply make sure to pass it the right input such that the output is canonical. func (s *Server) getTarget(ctx context.Context, ss *models.StorageStats, req models.Req) (out models.Series, err error) { @@ -352,6 +352,8 @@ func (s *Server) getTarget(ctx context.Context, ss *models.StorageStats, req mod QueryTo: req.To, QueryCons: req.ConsReq, Consolidator: req.Consolidator, + QueryMDP: req.MaxPoints, + QueryPNGroup: req.PNGroup, Meta: []models.SeriesMetaProperties{ { // note that for simplicity, we pretend that a read of rollup avg data is a read of 1 "avg series" @@ -405,7 +407,7 @@ func logLoad(typ string, key schema.AMKey, from, to uint32) { log.Debugf("DP load from %-6s %20s %d - %d (%s - %s) span:%ds", typ, key, from, to, util.TS(from), util.TS(to), to-from-1) } -// getSeriesFixed fetches the series and returns it in quantized, pre-canonical form. +// getSeriesFixed fetches the series and returns it in quantized, pre-canonical form with respect to their OutInterval // TODO: we can probably forego Fix if archive > 0, because only raw chunks are not quantized yet. // the requested consolidator is the one that will be used for selecting the archive to read from func (s *Server) getSeriesFixed(ctx context.Context, ss *models.StorageStats, req models.Req, consolidator consolidation.Consolidator) ([]schema.Point, error) { @@ -617,17 +619,19 @@ func (s *Server) getSeriesCachedStore(ctx *requestContext, ss *models.StorageSta return iters, nil } -// check for duplicate series names for the same query. If found merge the results. +// check for duplicate series names for the same query target. If found merge the results. // each first uniquely-identified series's backing datapoints slice is reused // any subsequent non-uniquely-identified series is merged into the former and has its // datapoints slice returned to the pool. input series must be canonical func mergeSeries(in []models.Series) []models.Series { type segment struct { - target string - query string - from uint32 - to uint32 - con consolidation.Consolidator + target string + query string + from uint32 + to uint32 + con consolidation.Consolidator + mdp uint32 + pngroup models.PNGroup } seriesByTarget := make(map[segment][]models.Series) for _, series := range in { @@ -637,6 +641,8 @@ func mergeSeries(in []models.Series) []models.Series { series.QueryFrom, series.QueryTo, series.Consolidator, + series.QueryMDP, + series.QueryPNGroup, } seriesByTarget[s] = append(seriesByTarget[s], series) } @@ -767,7 +773,7 @@ func newRequestContext(ctx context.Context, req *models.Req, consolidator consol // if the series has some excess at the end, it may aggregate into a bucket with a timestamp out of the desired range. // for example: imagine we take the case from above, and the user specified a `to` of 115. // a native 30s series would end with point 90. We should not include any points that would go into an aggregation bucket with timestamp higher than 90. - // (such as 100 or 110 which would technically be allowed by the `to` specification) + // (such as 100 or 110 which would technically be allowed by the `to` specification but land in the bucket with ts=120 which is out of bounds) // so the proper to value is the highest value that does not result in points going into an out-of-bounds bucket. // example: for 10s data (note that the 2 last colums should always match!) diff --git a/api/dataprocessor_test.go b/api/dataprocessor_test.go index 8d09895280..291038ee3d 100644 --- a/api/dataprocessor_test.go +++ b/api/dataprocessor_test.go @@ -390,7 +390,7 @@ func TestGetSeriesFixed(t *testing.T) { metric.Add(20+offset, 30) // this point will always be quantized to 30, so it should be selected metric.Add(30+offset, 40) // this point will always be quantized to 40 metric.Add(40+offset, 50) // this point will always be quantized to 50 - req := models.NewReq(id, "", "", from, to, 1000, 10, consolidation.Avg, 0, cluster.Manager.ThisNode(), 0, 0) + req := models.NewReq(id, "", "", from, to, 1000, 10, 0, consolidation.Avg, 0, cluster.Manager.ThisNode(), 0, 0) req.Archive = 0 req.ArchInterval = 10 points, err := srv.getSeriesFixed(test.NewContext(), &models.StorageStats{}, req, consolidation.None) @@ -582,7 +582,7 @@ func TestGetSeriesFixedVariableOutInterval(t *testing.T) { metric.Add(dataPoint.Ts, dataPoint.Val) } - req := models.NewReq(id, "", "", testCase.from, testCase.to, 1000, testCase.archInterval, consolidation.Avg, 0, cluster.Manager.ThisNode(), 0, 0) + req := models.NewReq(id, "", "", testCase.from, testCase.to, 1000, testCase.archInterval, 0, consolidation.Avg, 0, cluster.Manager.ThisNode(), 0, 0) req.Archive = 0 req.ArchInterval = testCase.archInterval req.OutInterval = testCase.outInterval @@ -598,19 +598,10 @@ func TestGetSeriesFixedVariableOutInterval(t *testing.T) { } func reqRaw(key schema.MKey, from, to, maxPoints, rawInterval uint32, consolidator consolidation.Consolidator, schemaId, aggId uint16) models.Req { - req := models.NewReq(key, "", "", from, to, maxPoints, rawInterval, consolidator, 0, cluster.Manager.ThisNode(), schemaId, aggId) + req := models.NewReq(key, "", "", from, to, maxPoints, rawInterval, 0, consolidator, 0, cluster.Manager.ThisNode(), schemaId, aggId) req.Archive = 0 return req } -func reqOut(key schema.MKey, from, to, maxPoints, rawInterval uint32, consolidator consolidation.Consolidator, schemaId, aggId uint16, archive uint8, archInterval, ttl, outInterval, aggNum uint32) models.Req { - req := models.NewReq(key, "", "", from, to, maxPoints, rawInterval, consolidator, 0, cluster.Manager.ThisNode(), schemaId, aggId) - req.Archive = archive - req.ArchInterval = archInterval - req.TTL = ttl - req.OutInterval = outInterval - req.AggNum = aggNum - return req -} func TestMergeSeries(t *testing.T) { out := make([]models.Series, 0) diff --git a/api/graphite.go b/api/graphite.go index c19fd4a155..460bd85060 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -225,7 +225,13 @@ func (s *Server) renderMetrics(ctx *middleware.Context, request models.GraphiteR // as graphite needs high-res data to perform its processing. mdp = 0 } - plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, nil) + + opts, err := optimizations.ApplyUserPrefs(request.Optimizations) + if err != nil { + response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) + return + } + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, opts) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { if request.NoProxy { @@ -661,7 +667,7 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) minFrom := uint32(math.MaxUint32) var maxTo uint32 - var reqs []models.Req + reqs := NewReqMap() metaTagEnrichmentData := make(map[string]tagquery.Tags) // note that different patterns to query can have different from / to, so they require different index lookups @@ -686,7 +692,7 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) if err != nil { return nil, meta, err } - series, err = s.clusterFindByTag(ctx, orgId, exprs, int64(r.From), maxSeriesPerReq-len(reqs), false) + series, err = s.clusterFindByTag(ctx, orgId, exprs, int64(r.From), maxSeriesPerReq-int(reqs.cnt), false) } else { series, err = s.findSeries(ctx, orgId, []string{r.Query}, int64(r.From)) } @@ -718,9 +724,9 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) cons = closestAggMethod(consReq, mdata.Aggregations.Get(archive.AggId).AggregationMethod) } - newReq := models.NewReq( - archive.Id, archive.NameWithTags(), r.Query, r.From, r.To, plan.MaxDataPoints, uint32(archive.Interval), cons, consReq, s.Node, archive.SchemaId, archive.AggId) - reqs = append(reqs, newReq) + newReq := r.ToModel() + newReq.Init(archive, cons, s.Node) + reqs.Add(newReq) } if tagquery.MetaTagSupport && len(metric.Defs) > 0 && len(metric.MetaTags) > 0 { @@ -739,31 +745,35 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) default: } - reqRenderSeriesCount.Value(len(reqs)) - if len(reqs) == 0 { + reqRenderSeriesCount.ValueUint32(reqs.cnt) + if reqs.cnt == 0 { return nil, meta, nil } - meta.RenderStats.SeriesFetch = uint32(len(reqs)) + meta.RenderStats.SeriesFetch = reqs.cnt // note: if 1 series has a movingAvg that requires a long time range extension, it may push other reqs into another archive. can be optimized later var err error - reqs, meta.RenderStats.PointsFetch, meta.RenderStats.PointsReturn, err = alignRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs) + var rp *ReqsPlan + rp, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints, maxPointsPerReqSoft, maxPointsPerReqHard) if err != nil { - log.Errorf("HTTP Render alignReq error: %s", err.Error()) return nil, meta, err } + meta.RenderStats.PointsFetch = rp.PointsFetch() + meta.RenderStats.PointsReturn = rp.PointsReturn(plan.MaxDataPoints) + reqsList := rp.List() + span := opentracing.SpanFromContext(ctx) - span.SetTag("num_reqs", len(reqs)) + span.SetTag("num_reqs", len(reqsList)) span.SetTag("points_fetch", meta.RenderStats.PointsFetch) span.SetTag("points_return", meta.RenderStats.PointsReturn) - for _, req := range reqs { + for _, req := range reqsList { log.Debugf("HTTP Render %s - arch:%d archI:%d outI:%d aggN: %d from %s", req, req.Archive, req.ArchInterval, req.OutInterval, req.AggNum, req.Node.GetName()) } a := time.Now() - out, err := s.getTargets(ctx, &meta.StorageStats, reqs) + out, err := s.getTargets(ctx, &meta.StorageStats, reqsList) if err != nil { log.Errorf("HTTP Render %s", err.Error()) return nil, meta, err @@ -787,7 +797,7 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) data := make(map[expr.Req][]models.Series) for _, serie := range out { - q := expr.NewReq(serie.QueryPatt, serie.QueryFrom, serie.QueryTo, serie.QueryCons) + q := expr.NewReqFromSerie(serie) data[q] = append(data[q], serie) } @@ -1415,7 +1425,12 @@ func (s *Server) showPlan(ctx *middleware.Context, request models.GraphiteRender stable := request.Process == "stable" mdp := request.MaxDataPoints - plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, nil) + opts, err := optimizations.ApplyUserPrefs(request.Optimizations) + if err != nil { + response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) + return + } + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, opts) if err != nil { response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) return diff --git a/api/graphite_req.go b/api/graphite_req.go new file mode 100644 index 0000000000..5dbb7bbc3d --- /dev/null +++ b/api/graphite_req.go @@ -0,0 +1,167 @@ +package api + +import ( + "fmt" + + "github.com/grafana/metrictank/api/models" +) + +// ReqMap is a map of requests of data, +// it has single requests for which no pre-normalization effort will be performed, and +// requests that can be pre-normalized together to the same resolution, bundled by their PNGroup +type ReqMap struct { + single []models.Req + pngroups map[models.PNGroup][]models.Req + cnt uint32 +} + +func NewReqMap() *ReqMap { + return &ReqMap{ + pngroups: make(map[models.PNGroup][]models.Req), + } +} + +// Add adds a models.Req to the ReqMap +func (r *ReqMap) Add(req models.Req) { + r.cnt++ + if req.PNGroup == 0 { + r.single = append(r.single, req) + return + } + r.pngroups[req.PNGroup] = append(r.pngroups[req.PNGroup], req) +} + +// Dump provides a human readable string representation of the ReqsMap +func (r ReqMap) Dump() string { + out := fmt.Sprintf("ReqsMap (%d entries):\n", r.cnt) + out += " Groups:\n" + for i, reqs := range r.pngroups { + out += fmt.Sprintf(" * group %d:", i) + for _, r := range reqs { + out += " " + r.DebugString() + "\n" + } + } + out += " Single:\n" + for _, r := range r.single { + out += " " + r.DebugString() + "\n" + } + return out +} + +// PNGroupSplit embodies a PNGroup broken down by whether requests are MDP-optimizable +type PNGroupSplit struct { + mdpyes []models.Req // MDP-optimizable requests + mdpno []models.Req // not MDP-optimizable reqs +} + +// ReqsPlan holds requests that have been planned, broken down by PNGroup and MDP-optimizability +type ReqsPlan struct { + pngroups map[models.PNGroup]PNGroupSplit + single PNGroupSplit + cnt uint32 +} + +// NewReqsPlan generates a ReqsPlan based on the provided ReqMap. +func NewReqsPlan(reqs ReqMap) ReqsPlan { + rp := ReqsPlan{ + pngroups: make(map[models.PNGroup]PNGroupSplit), + cnt: reqs.cnt, + } + for group, groupReqs := range reqs.pngroups { + var split PNGroupSplit + for _, req := range groupReqs { + if req.MaxPoints > 0 { + split.mdpyes = append(split.mdpyes, req) + } else { + split.mdpno = append(split.mdpno, req) + } + } + rp.pngroups[group] = split + } + for _, req := range reqs.single { + if req.MaxPoints > 0 { + rp.single.mdpyes = append(rp.single.mdpyes, req) + } else { + rp.single.mdpno = append(rp.single.mdpno, req) + } + } + return rp +} + +// PointsFetch returns how many points this plan will fetch when executed +func (rp ReqsPlan) PointsFetch() uint32 { + var cnt uint32 + for _, r := range rp.single.mdpyes { + cnt += r.PointsFetch() + } + for _, r := range rp.single.mdpno { + cnt += r.PointsFetch() + } + for _, split := range rp.pngroups { + for _, r := range split.mdpyes { + cnt += r.PointsFetch() + } + for _, r := range split.mdpno { + cnt += r.PointsFetch() + } + } + return cnt +} + +// Dump provides a human readable string representation of the ReqsPlan +func (rp ReqsPlan) Dump() string { + out := fmt.Sprintf("ReqsPlan (%d entries):\n", rp.cnt) + out += " Groups:\n" + for i, split := range rp.pngroups { + out += fmt.Sprintf(" * group %d\nMDP-yes:\n", i) + for _, r := range split.mdpyes { + out += " " + r.DebugString() + "\n" + } + out += " MDP-no:\n" + for _, r := range split.mdpno { + out += " " + r.DebugString() + "\n" + } + } + out += " Single MDP-yes:\n" + for _, r := range rp.single.mdpyes { + out += " " + r.DebugString() + "\n" + } + out += " Single MDP-no:\n" + for _, r := range rp.single.mdpno { + out += " " + r.DebugString() + "\n" + } + return out +} + +// PointsReturn estimates the amount of points that will be returned for this request +// best effort: not aware of summarize(), aggregation functions, runtime normalization. but does account for runtime consolidation +func (rp ReqsPlan) PointsReturn(planMDP uint32) uint32 { + var cnt uint32 + for _, r := range rp.single.mdpyes { + cnt += r.PointsReturn(planMDP) + } + for _, r := range rp.single.mdpno { + cnt += r.PointsReturn(planMDP) + } + for _, split := range rp.pngroups { + for _, r := range split.mdpyes { + cnt += r.PointsReturn(planMDP) + } + for _, r := range split.mdpno { + cnt += r.PointsReturn(planMDP) + } + } + return cnt +} + +// List returns the requests contained within the plan as a slice +func (rp ReqsPlan) List() []models.Req { + l := make([]models.Req, 0, rp.cnt) + l = append(l, rp.single.mdpno...) + l = append(l, rp.single.mdpyes...) + for _, g := range rp.pngroups { + l = append(l, g.mdpno...) + l = append(l, g.mdpyes...) + } + return l +} diff --git a/api/models/graphite.go b/api/models/graphite.go index a1fd4f42cd..5d39511769 100644 --- a/api/models/graphite.go +++ b/api/models/graphite.go @@ -55,6 +55,7 @@ type GraphiteRender struct { NoProxy bool `json:"local" form:"local"` //this is set to true by graphite-web when it passes request to cluster servers Meta bool `json:"meta" form:"meta"` // request for meta data, which will be returned as long as the format is compatible (json) and we don't have to go via graphite Process string `json:"process" form:"process" binding:"In(,none,stable,any);Default(stable)"` + Optimizations string `json:"optimizations" form:"optimizations"` } func (gr GraphiteRender) Validate(ctx *macaron.Context, errs binding.Errors) binding.Errors { diff --git a/api/models/request.go b/api/models/request.go index 997d7e42e6..9c3b98443c 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -3,6 +3,8 @@ package models import ( "fmt" + "github.com/grafana/metrictank/conf" + "github.com/grafana/metrictank/idx" "github.com/grafana/metrictank/schema" "github.com/grafana/metrictank/cluster" @@ -12,6 +14,9 @@ import ( "github.com/opentracing/opentracing-go/log" ) +//go:generate msgp +//msgp:ignore Req + // Req is a request for data by MKey and parameters such as consolidator, max points, etc type Req struct { // these fields can be set straight away: @@ -21,8 +26,9 @@ type Req struct { From uint32 `json:"from"` To uint32 `json:"to"` MaxPoints uint32 `json:"maxPoints"` + PNGroup PNGroup `json:"pngroup"` RawInterval uint32 `json:"rawInterval"` // the interval of the raw metric before any consolidation - // the consolidation method for rollup archive and normalization. (not runtime consolidation) + // the consolidation method for rollup archive and normalization (pre-normalization and runtime normalization). (but not runtime consolidation) // ConsReq 0 -> configured value // Conseq != 0 -> closest value we can offer based on config Consolidator consolidation.Consolidator `json:"consolidator"` @@ -35,7 +41,7 @@ type Req struct { SchemaId uint16 `json:"schemaId"` AggId uint16 `json:"aggId"` - // these fields need some more coordination and are typically set later (after request alignment) + // these fields need some more coordination and are typically set later (after request planning) Archive uint8 `json:"archive"` // 0 means original data, 1 means first agg level, 2 means 2nd, etc. ArchInterval uint32 `json:"archInterval"` // the interval corresponding to the archive we'll fetch TTL uint32 `json:"ttl"` // the ttl of the archive we'll fetch @@ -43,8 +49,11 @@ type Req struct { AggNum uint32 `json:"aggNum"` // how many points to consolidate together at runtime, after fetching from the archive (normalization) } -// NewReq creates a new request. It sets all properties minus the ones that need request alignment -func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterval uint32, cons, consReq consolidation.Consolidator, node cluster.Node, schemaId, aggId uint16) Req { +// PNGroup is an identifier for a pre-normalization group: data that can be pre-normalized together +type PNGroup uint64 + +// NewReq creates a new request. It sets all properties except the ones that need request planning +func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterval uint32, pngroup PNGroup, cons, consReq consolidation.Consolidator, node cluster.Node, schemaId, aggId uint16) Req { return Req{ MKey: key, Target: target, @@ -52,6 +61,7 @@ func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterv From: from, To: to, MaxPoints: maxPoints, + PNGroup: pngroup, RawInterval: rawInterval, Consolidator: cons, ConsReq: consReq, @@ -61,13 +71,91 @@ func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterv } } +// Init initializes a request based on the metadata that we know of. +// It sets all properties except the ones that need request planning +func (r *Req) Init(archive idx.Archive, cons consolidation.Consolidator, node cluster.Node) { + r.MKey = archive.Id + r.Target = archive.NameWithTags() + r.RawInterval = uint32(archive.Interval) + r.Consolidator = cons + r.Node = node + r.SchemaId = archive.SchemaId + r.AggId = archive.AggId +} + +// Plan updates the planning parameters to match the i'th archive in its retention rules +func (r *Req) Plan(i int, ret conf.Retention) { + r.Archive = uint8(i) + if i == 0 { + // The first retention is raw data, so use its native interval + r.ArchInterval = r.RawInterval + } else { + r.ArchInterval = uint32(ret.SecondsPerPoint) + } + r.TTL = uint32(ret.MaxRetention()) + r.OutInterval = r.ArchInterval + r.AggNum = 1 +} + +// PlanNormalization updates the planning parameters to accommodate normalization to the specified interval +func (r *Req) PlanNormalization(interval uint32) { + r.OutInterval = interval + r.AggNum = interval / r.ArchInterval +} + +// AdjustTo adjusts the request to accommodate the requested interval +// notes: +// * the Req MUST have been Plan()'d already! +// * interval MUST be a multiple of the ArchInterval (so we can normalize if needed) +// * the TTL of lower resolution archives is always assumed to be at least as long as the current archive +func (r *Req) AdjustTo(interval, from uint32, rets []conf.Retention) { + + // if we satisfy the interval with our current settings, nothing left to do + if r.ArchInterval == interval { + return + } + + // let's see if we can deliver it via a lower-res rollup archive. + for i, ret := range rets[r.Archive+1:] { + if interval == uint32(ret.SecondsPerPoint) && ret.Ready <= from { + // we're in luck. this will be more efficient than runtime consolidation + r.Plan(int(r.Archive)+1+i, ret) + return + } + } + + // we will have to apply normalization + // we use the initially found archive as starting point. there could be some cases - if you have exotic settings - + // where it may be more efficient to pick a lower res archive as starting point (it would still require an interval + // that is a factor of the output interval) but let's not worry about that edge case. + r.PlanNormalization(interval) +} + +// PointsFetch returns how many points this request will fetch when executed +func (r Req) PointsFetch() uint32 { + return (r.To - r.From) / r.ArchInterval +} + +// PointsReturn estimates the amount of points that will be returned for this request +// best effort: not aware of summarize(), runtime normalization. but does account for runtime consolidation +func (r Req) PointsReturn(planMDP uint32) uint32 { + points := (r.To - r.From) / r.OutInterval + if planMDP > 0 && points > planMDP { + // note that we don't assign to req.AggNum here, because that's only for normalization. + // MDP runtime consolidation doesn't look at req.AggNum + aggNum := consolidation.AggEvery(points, planMDP) + points /= aggNum + } + return points +} + func (r Req) String() string { return fmt.Sprintf("%s %d - %d (%s - %s) span:%ds. points <= %d. %s.", r.MKey.String(), r.From, r.To, util.TS(r.From), util.TS(r.To), r.To-r.From-1, r.MaxPoints, r.Consolidator) } func (r Req) DebugString() string { - return fmt.Sprintf("Req key=%q target=%q pattern=%q %d - %d (%s - %s) (span %d) maxPoints=%d rawInt=%d cons=%s consReq=%d schemaId=%d aggId=%d archive=%d archInt=%d ttl=%d outInt=%d aggNum=%d", - r.MKey, r.Target, r.Pattern, r.From, r.To, util.TS(r.From), util.TS(r.To), r.To-r.From-1, r.MaxPoints, r.RawInterval, r.Consolidator, r.ConsReq, r.SchemaId, r.AggId, r.Archive, r.ArchInterval, r.TTL, r.OutInterval, r.AggNum) + return fmt.Sprintf("Req key=%q target=%q pattern=%q %d - %d (%s - %s) (span %d) maxPoints=%d pngroup=%d rawInt=%d cons=%s consReq=%d schemaId=%d aggId=%d archive=%d archInt=%d ttl=%d outInt=%d aggNum=%d", + r.MKey, r.Target, r.Pattern, r.From, r.To, util.TS(r.From), util.TS(r.To), r.To-r.From-1, r.MaxPoints, r.PNGroup, r.RawInterval, r.Consolidator, r.ConsReq, r.SchemaId, r.AggId, r.Archive, r.ArchInterval, r.TTL, r.OutInterval, r.AggNum) } // TraceLog puts all request properties in a span log entry @@ -83,6 +171,7 @@ func (r Req) TraceLog(span opentracing.Span) { log.Uint32("to", r.To), log.Uint32("span", r.To-r.From-1), log.Uint32("mdp", r.MaxPoints), + log.Uint64("pngroup", uint64(r.PNGroup)), log.Uint32("rawInterval", r.RawInterval), log.String("cons", r.Consolidator.String()), log.String("consReq", r.ConsReq.String()), @@ -97,12 +186,7 @@ func (r Req) TraceLog(span opentracing.Span) { } // Equals compares all fields of a to b for equality. -// Except -// * TTL (because alignRequests may change it) -// for 100% correctness we may want to fix this in the future -// but for now, should be harmless since the field is not -// that important for archive fetching -// * For the Node field we just compare the node.Name +// Except the Node field: we just compare the node.Name // rather then doing a deep comparison. func (a Req) Equals(b Req) bool { if a.MKey != b.MKey { @@ -123,6 +207,9 @@ func (a Req) Equals(b Req) bool { if a.MaxPoints != b.MaxPoints { return false } + if a.PNGroup != b.PNGroup { + return false + } if a.RawInterval != b.RawInterval { return false } @@ -150,6 +237,9 @@ func (a Req) Equals(b Req) bool { if a.OutInterval != b.OutInterval { return false } + if a.TTL != b.TTL { + return false + } if a.AggNum != b.AggNum { return false } diff --git a/api/models/request_gen.go b/api/models/request_gen.go new file mode 100644 index 0000000000..c6231ca2cd --- /dev/null +++ b/api/models/request_gen.go @@ -0,0 +1,59 @@ +package models + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. + +import ( + "github.com/tinylib/msgp/msgp" +) + +// DecodeMsg implements msgp.Decodable +func (z *PNGroup) DecodeMsg(dc *msgp.Reader) (err error) { + { + var zb0001 uint64 + zb0001, err = dc.ReadUint64() + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = PNGroup(zb0001) + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z PNGroup) EncodeMsg(en *msgp.Writer) (err error) { + err = en.WriteUint64(uint64(z)) + if err != nil { + err = msgp.WrapError(err) + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z PNGroup) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + o = msgp.AppendUint64(o, uint64(z)) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *PNGroup) UnmarshalMsg(bts []byte) (o []byte, err error) { + { + var zb0001 uint64 + zb0001, bts, err = msgp.ReadUint64Bytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + (*z) = PNGroup(zb0001) + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z PNGroup) Msgsize() (s int) { + s = msgp.Uint64Size + return +} diff --git a/api/models/request_gen_test.go b/api/models/request_gen_test.go new file mode 100644 index 0000000000..f33cd6ee64 --- /dev/null +++ b/api/models/request_gen_test.go @@ -0,0 +1,3 @@ +package models + +// Code generated by github.com/tinylib/msgp DO NOT EDIT. diff --git a/api/models/series.go b/api/models/series.go index 59b7ebeafb..d99b0cb708 100644 --- a/api/models/series.go +++ b/api/models/series.go @@ -26,6 +26,8 @@ type Series struct { QueryTo uint32 // to tie series back to request it came from QueryCons consolidation.Consolidator // to tie series back to request it came from (may be 0 to mean use configured default) Consolidator consolidation.Consolidator // consolidator to actually use (for fetched series this may not be 0, default must be resolved. if series created by function, may be 0) + QueryMDP uint32 // to tie series back to request it came from + QueryPNGroup PNGroup // to tie series back to request it came from Meta SeriesMeta // note: this series could be a "just fetched" series, or one derived from many other series Datapoints []schema.Point } @@ -77,7 +79,7 @@ func (smp SeriesMetaProperties) Export() SeriesMetaPropertiesExport { } } -// Merge merges SeriesMeta b into a +// Merge merges SeriesMeta b into a and returns the modified a // counts for identical properties get added together func (a SeriesMeta) Merge(b SeriesMeta) SeriesMeta { // note: to see which properties are equivalent we should not consider the count @@ -209,6 +211,8 @@ func (s Series) Copy(emptyDatapoints []schema.Point) Series { QueryTo: s.QueryTo, QueryCons: s.QueryCons, Consolidator: s.Consolidator, + QueryMDP: s.QueryMDP, + QueryPNGroup: s.QueryPNGroup, Meta: s.Meta.Copy(), } } diff --git a/api/models/series_gen.go b/api/models/series_gen.go index 54823c83cb..25215a9163 100644 --- a/api/models/series_gen.go +++ b/api/models/series_gen.go @@ -97,6 +97,18 @@ func (z *Series) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "Consolidator") return } + case "QueryMDP": + z.QueryMDP, err = dc.ReadUint32() + if err != nil { + err = msgp.WrapError(err, "QueryMDP") + return + } + case "QueryPNGroup": + err = z.QueryPNGroup.DecodeMsg(dc) + if err != nil { + err = msgp.WrapError(err, "QueryPNGroup") + return + } case "Meta": var zb0003 uint32 zb0003, err = dc.ReadArrayHeader() @@ -148,9 +160,9 @@ func (z *Series) DecodeMsg(dc *msgp.Reader) (err error) { // EncodeMsg implements msgp.Encodable func (z *Series) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 10 + // map header, size 12 // write "Target" - err = en.Append(0x8a, 0xa6, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74) + err = en.Append(0x8c, 0xa6, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74) if err != nil { return } @@ -241,6 +253,26 @@ func (z *Series) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "Consolidator") return } + // write "QueryMDP" + err = en.Append(0xa8, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x44, 0x50) + if err != nil { + return + } + err = en.WriteUint32(z.QueryMDP) + if err != nil { + err = msgp.WrapError(err, "QueryMDP") + return + } + // write "QueryPNGroup" + err = en.Append(0xac, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x4e, 0x47, 0x72, 0x6f, 0x75, 0x70) + if err != nil { + return + } + err = z.QueryPNGroup.EncodeMsg(en) + if err != nil { + err = msgp.WrapError(err, "QueryPNGroup") + return + } // write "Meta" err = en.Append(0xa4, 0x4d, 0x65, 0x74, 0x61) if err != nil { @@ -281,9 +313,9 @@ func (z *Series) EncodeMsg(en *msgp.Writer) (err error) { // MarshalMsg implements msgp.Marshaler func (z *Series) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 10 + // map header, size 12 // string "Target" - o = append(o, 0x8a, 0xa6, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74) + o = append(o, 0x8c, 0xa6, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74) o = msgp.AppendString(o, z.Target) // string "Tags" o = append(o, 0xa4, 0x54, 0x61, 0x67, 0x73) @@ -318,6 +350,16 @@ func (z *Series) MarshalMsg(b []byte) (o []byte, err error) { err = msgp.WrapError(err, "Consolidator") return } + // string "QueryMDP" + o = append(o, 0xa8, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x44, 0x50) + o = msgp.AppendUint32(o, z.QueryMDP) + // string "QueryPNGroup" + o = append(o, 0xac, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x4e, 0x47, 0x72, 0x6f, 0x75, 0x70) + o, err = z.QueryPNGroup.MarshalMsg(o) + if err != nil { + err = msgp.WrapError(err, "QueryPNGroup") + return + } // string "Meta" o = append(o, 0xa4, 0x4d, 0x65, 0x74, 0x61) o = msgp.AppendArrayHeader(o, uint32(len(z.Meta))) @@ -431,6 +473,18 @@ func (z *Series) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "Consolidator") return } + case "QueryMDP": + z.QueryMDP, bts, err = msgp.ReadUint32Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "QueryMDP") + return + } + case "QueryPNGroup": + bts, err = z.QueryPNGroup.UnmarshalMsg(bts) + if err != nil { + err = msgp.WrapError(err, "QueryPNGroup") + return + } case "Meta": var zb0003 uint32 zb0003, bts, err = msgp.ReadArrayHeaderBytes(bts) @@ -490,7 +544,7 @@ func (z *Series) Msgsize() (s int) { s += msgp.StringPrefixSize + len(za0001) + msgp.StringPrefixSize + len(za0002) } } - s += 9 + msgp.Uint32Size + 10 + msgp.StringPrefixSize + len(z.QueryPatt) + 10 + msgp.Uint32Size + 8 + msgp.Uint32Size + 10 + z.QueryCons.Msgsize() + 13 + z.Consolidator.Msgsize() + 5 + msgp.ArrayHeaderSize + s += 9 + msgp.Uint32Size + 10 + msgp.StringPrefixSize + len(z.QueryPatt) + 10 + msgp.Uint32Size + 8 + msgp.Uint32Size + 10 + z.QueryCons.Msgsize() + 13 + z.Consolidator.Msgsize() + 9 + msgp.Uint32Size + 13 + z.QueryPNGroup.Msgsize() + 5 + msgp.ArrayHeaderSize for za0003 := range z.Meta { s += z.Meta[za0003].Msgsize() } diff --git a/api/query_engine.go b/api/query_engine.go index 7950336be0..719c541753 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -2,10 +2,12 @@ package api import ( "math" + "net/http" + "reflect" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/api/response" - "github.com/grafana/metrictank/consolidation" + "github.com/grafana/metrictank/conf" "github.com/grafana/metrictank/mdata" "github.com/grafana/metrictank/stats" "github.com/grafana/metrictank/util" @@ -17,142 +19,339 @@ var ( reqRenderChosenArchive = stats.NewMeter32("api.request.render.chosen_archive", false) // metric api.request.render.points_fetched is the number of points that need to be fetched for a /render request. reqRenderPointsFetched = stats.NewMeter32("api.request.render.points_fetched", false) - // metric api.request.render.points_returned is the number of points the request will return. + // metric api.request.render.points_returned is the number of points the request will return + // best effort: not aware of summarize(), aggregation functions, runtime normalization. but does account for runtime consolidation reqRenderPointsReturned = stats.NewMeter32("api.request.render.points_returned", false) - errUnSatisfiable = response.NewError(404, "request cannot be satisfied due to lack of available retentions") - errMaxPointsPerReq = response.NewError(413, "request exceeds max-points-per-req-hard limit. Reduce the time range or number of targets or ask your admin to increase the limit.") + errUnSatisfiable = response.NewError(http.StatusNotFound, "request cannot be satisfied due to lack of available retentions") + errMaxPointsPerReq = response.NewError(http.StatusRequestEntityTooLarge, "request exceeds max-points-per-req-hard limit. Reduce the time range or number of targets or ask your admin to increase the limit.") ) -// alignRequests updates the requests with all details for fetching, making sure all metrics are in the same, optimal interval -// note: it is assumed that all requests have the same maxDataPoints, from & to. +func getRetentions(req models.Req) []conf.Retention { + return mdata.Schemas.Get(req.SchemaId).Retentions.Rets +} + +// planRequests updates the requests with all details for fetching. +// Notes: +// [1] MDP-optimization may reduce amount of points down to MDP/2, but not lower. TODO: how about reduce to MDP exactly if possible, and a bit lower otherwise +// Typically MDP matches number of pixels, which is very dense. So MDP/2 is still quite dense, and for our purposes we consider MDP/2 points to contain the same amount of "information". +// [2] MDP-optimizable requests (when considered by themselves) incur no significant information loss. See [1] +// Though consider this case: +// series A 10s:7d,5min:70d +// series B 10s:7d,4min:30d +// Let's say a request comes in for 2 days worth of data with MDP=800. Using the high-res data would be 17280 points and require runtime consolidation +// Both series can be MDP optimized: pick archive=1 and get 576 and 720 points respectively. Neither lost information. +// However, if it then turns out that both series need to be combined in an aggregation function, they need to be reduced to 20 min resolution, which results in coarse points. +// Thus MDP-optimized series can still possibly result in some information loss, though this seems quite rare. +// If you want to aggregate different data together, just give it compatible intervals. For our purposes we will consider MDP-optimizing safe. +// [3] Requests in the same PNGroup will need to be normalized together anyway. +// Because the consolidation function for normalization is always set taking into account the rollups that we have (see executePlan()) we can better read from a coarser archive. +// Any request in a PNGroup has already been vetted to be worthy of pre-normalization, thus there is absolutely no loss of information. +// +// planRequests follows these steps: +// 1) Initial parameters. +// select the highest resolution possible within TTL for all requests. there's 4 cases: +// * requests in the same PNGroup, and MDP-optimizable: reduce aggressively: to longest common interval such that points >=MDP/2 +// * requests in the same PNGroup but not MDP-optimizable: reduce conservatively: to shortest common interval that still meets TTL +// * MDP optimizable singles : longest interval such that points >= MDP/2 +// * non-MDP-optimizable singles : shortest interval that still meets TTL +// +// 2) apply max-points-per-req-soft (meaning: pick coarser data as needed) +// The optimizations in the previous step should increase the odds of meeting this limit. +// If we still breach this limit, we could +// a) reduce the already MDP-optimized ones further but that would definitely result in loss of accuracy +// b) reduce non-MDP-optimizable series. +// For "fairness" across series, and because we used to simply reduce any series without regard for how it would be used, we pick the latter. better would be both +// 3) subject to max-points-per-req-hard: reject the query if it can't be met +// +// note: it is assumed that all requests have the same from & to. // also takes a "now" value which we compare the TTL against -func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint32, uint32, error) { - tsRange := to - from +func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32, mpprSoft, mpprHard int) (*ReqsPlan, error) { - var listIntervals []uint32 - var seenIntervals = make(map[uint32]struct{}) - var targets = make(map[string]struct{}) + ok, rp := false, NewReqsPlan(*reqs) - for _, req := range reqs { - targets[req.Target] = struct{}{} + for group, split := range rp.pngroups { + if len(split.mdpno) > 0 { + split.mdpno, ok = planHighestResMulti(now, from, to, split.mdpno) + if !ok { + return nil, errUnSatisfiable + } + } + if len(split.mdpyes) > 0 { + split.mdpyes, ok = planLowestResForMDPMulti(now, from, to, split.mdpyes) + if !ok { + return nil, errUnSatisfiable + } + rp.pngroups[group] = split + } + } + for i, req := range rp.single.mdpno { + rp.single.mdpno[i], ok = planHighestResSingle(now, from, to, req) + if !ok { + return nil, errUnSatisfiable + } + } + for i, req := range rp.single.mdpyes { + rp.single.mdpyes[i], ok = planLowestResForMDPSingle(now, from, to, req) + if !ok { + return nil, errUnSatisfiable + } + } + + if mpprSoft > 0 { + // at this point, all MDP-optimizable series have already been optimized + // we can try to reduce the resolution of non-MDP-optimizable series + // if metrictank is already handling all, or most of your queries, then we have been able to determine + // MDP-optimizability very well. If the request came from Graphite, we have to assume it may run GR-functions. + // thus in the former case, we pretty much know that this is going to have an adverse effect on your queries, + // and you should probably not use this option, or we should even get rid of it. + // in the latter case though, it's quite likely we were too cautious and categorized many series as non-MDP + // optimizable whereas in reality they should be, so in that case this option is a welcome way to reduce the + // impact of big queries + // we could do two approaches: gradually reduce the interval of all series/groups being read, or just aggressively + // adjust one group at a time. The latter seems simpler, so for now we do just that. + if rp.PointsFetch() > uint32(mpprSoft) { + for group, split := range rp.pngroups { + if len(split.mdpno) > 0 { + split.mdpno, ok = planLowestResForMDPMulti(now, from, to, split.mdpno) + if !ok { + return nil, errUnSatisfiable + } + rp.pngroups[group] = split + if rp.PointsFetch() <= uint32(mpprSoft) { + goto HonoredSoft + } + } + } + for i, req := range rp.single.mdpno { + rp.single.mdpno[i], ok = planLowestResForMDPSingle(now, from, to, req) + if !ok { + return nil, errUnSatisfiable + } + // for every 10 requests we adjusted, check if we honor soft now. + // note that there may be thousands of requests + if i%10 == 9 { + if rp.PointsFetch() <= uint32(mpprSoft) { + goto HonoredSoft + } + } + } + } + } +HonoredSoft: + + if mpprHard > 0 && int(rp.PointsFetch()) > mpprHard { + return nil, errMaxPointsPerReq + + } + + // send out some metrics and we're done! + for _, r := range rp.single.mdpyes { + reqRenderChosenArchive.ValueUint32(uint32(r.Archive)) + } + for _, r := range rp.single.mdpno { + reqRenderChosenArchive.ValueUint32(uint32(r.Archive)) + } + for _, split := range rp.pngroups { + for _, r := range split.mdpyes { + reqRenderChosenArchive.ValueUint32(uint32(r.Archive)) + } + for _, r := range split.mdpno { + reqRenderChosenArchive.ValueUint32(uint32(r.Archive)) + } } - numTargets := uint32(len(targets)) - minTTL := now - reqs[0].From + reqRenderPointsFetched.ValueUint32(rp.PointsFetch()) + reqRenderPointsReturned.ValueUint32(rp.PointsReturn(planMDP)) - minIntervalSoft := uint32(0) - minIntervalHard := uint32(0) + return &rp, nil +} + +func planHighestResSingle(now, from, to uint32, req models.Req) (models.Req, bool) { + rets := getRetentions(req) + minTTL := now - from + var ok bool + for i, ret := range rets { + // skip non-ready option. + if ret.Ready > from { + continue + } + ok = true + req.Plan(i, ret) - if maxPointsPerReqSoft > 0 { - minIntervalSoft = uint32(math.Ceil(float64(tsRange) / (float64(maxPointsPerReqSoft) / float64(numTargets)))) + if req.TTL >= minTTL { + break + } } - if maxPointsPerReqHard > 0 { - minIntervalHard = uint32(math.Ceil(float64(tsRange) / (float64(maxPointsPerReqHard) / float64(numTargets)))) + return req, ok +} + +func planLowestResForMDPSingle(now, from, to uint32, req models.Req) (models.Req, bool) { + rets := getRetentions(req) + var ok bool + for i := len(rets) - 1; i >= 0; i-- { + // skip non-ready option. + if rets[i].Ready > from { + continue + } + ok = true + req.Plan(i, rets[i]) + if req.PointsFetch() >= req.MaxPoints/2 { + break + } } + return req, ok +} +func planHighestResMulti(now, from, to uint32, reqs []models.Req) ([]models.Req, bool) { + minTTL := now - from + + var listIntervals []uint32 + var seenIntervals = make(map[uint32]struct{}) - // set preliminary settings. may be adjusted further down - // but for now: - // for each req, find the highest res archive - // (starting with raw, then rollups in decreasing precision) - // that retains all the data we need and does not exceed minIntervalSoft. - // fallback to lowest res option (which *should* have the longest TTL) - var found bool for i := range reqs { req := &reqs[i] - retentions := mdata.Schemas.Get(req.SchemaId).Retentions.Rets - for i, ret := range retentions { + var ok bool + rets := getRetentions(*req) + for i, ret := range rets { // skip non-ready option. if ret.Ready > from { continue } - found = true - req.Archive = uint8(i) - req.TTL = uint32(ret.MaxRetention()) - if i == 0 { - // The first retention is raw data, so use its native interval - req.ArchInterval = req.RawInterval - } else { - req.ArchInterval = uint32(ret.SecondsPerPoint) - } + ok = true + req.Plan(i, ret) - if req.TTL >= minTTL && req.ArchInterval >= minIntervalSoft { + if req.TTL >= minTTL { break } } - if !found { - return nil, 0, 0, errUnSatisfiable + if !ok { + return nil, ok } - if _, ok := seenIntervals[req.ArchInterval]; !ok { listIntervals = append(listIntervals, req.ArchInterval) seenIntervals[req.ArchInterval] = struct{}{} } } - - // due to different retentions coming into play, different requests may end up with different resolutions - // we all need to emit them at the same interval, the LCM interval >= interval of the req interval := util.Lcm(listIntervals) - if interval < minIntervalHard { - return nil, 0, 0, errMaxPointsPerReq + // plan all our requests so that they result in the common output interval. + for i := range reqs { + req := &reqs[i] + req.AdjustTo(interval, from, getRetentions(*req)) } - // now, for all our requests, set all their properties. we may have to apply runtime consolidation to get the - // correct output interval if out interval != native. In that case, we also check whether we can fulfill - // the request by reading from an archive instead (i.e. whether it has the correct interval. - // the TTL of lower resolution archives is always assumed to be at least as long so we don't have to check that) + return reqs, true +} - var pointsFetch uint32 - for i := range reqs { - req := &reqs[i] - if req.ArchInterval == interval { - // the easy case. we can satisfy this req with what we already found - // just have to set a few more options - req.OutInterval = req.ArchInterval - req.AggNum = 1 - - } else { - // the harder case. due to other reqs with different retention settings - // we have to deliver an interval higher than what we originally came up with - - // let's see first if we can deliver it via lower-res rollup archives, if we have any - retentions := mdata.Schemas.Get(req.SchemaId).Retentions.Rets - for i, ret := range retentions[req.Archive+1:] { - archInterval := uint32(ret.SecondsPerPoint) - if interval == archInterval && ret.Ready <= from { - // we're in luck. this will be more efficient than runtime consolidation - req.Archive = req.Archive + 1 + uint8(i) - req.ArchInterval = archInterval - req.TTL = uint32(ret.MaxRetention()) - req.OutInterval = archInterval - req.AggNum = 1 - break - } +// note: we can assume all reqs have the same MDP. +func planLowestResForMDPMulti(now, from, to uint32, reqs []models.Req) ([]models.Req, bool) { + var ok bool + minTTL := now - from + + // if we were to set each req to their coarsest interval that results in >= MDP/2 points, + // we'd still have to align them to their LCM interval, which may push them in to + // "too coarse" territory. + // instead, we pick the coarsest allowable artificial interval... + maxInterval := (2 * (to - from)) / reqs[0].MaxPoints + // ...and then we look for the combination of intervals that scores highest. + // the bigger the interval the better (load less points), adjusted for number of reqs that + // have that interval. but their combined LCM may not exceed maxInterval. + + var validIntervalss [][]uint32 + // first, find the unique set of retentions we're dealing with. + retentions := make(map[uint16]struct{}) + for _, req := range reqs { + retentions[req.SchemaId] = struct{}{} + } + + // now, extract the set of valid intervals from each retention + // if a retention has no valid intervals, we can't satisfy the request + for schemaID := range retentions { + var ok bool + rets := mdata.Schemas.Get(schemaID).Retentions.Rets + var validIntervals []uint32 + for _, ret := range rets { + if ret.Ready <= from && ret.MaxRetention() >= int(minTTL) { + ok = true + validIntervals = append(validIntervals, uint32(ret.SecondsPerPoint)) } - if req.ArchInterval != interval { - // we have not been able to find an archive matching the desired output interval - // we will have to apply runtime consolidation - // we use the initially found archive as starting point. there could be some cases - if you have exotic settings - - // where it may be more efficient to pick a lower res archive as starting point (it would still require an interval - // divisible by the output interval) but let's not worry about that edge case. - req.OutInterval = interval - req.AggNum = interval / req.ArchInterval + } + if !ok { + return nil, ok + } + // add our sequence of valid intervals to the list, unless it's there already + var found bool + for _, v := range validIntervalss { + if reflect.DeepEqual(v, validIntervals) { + found = true + break } } - pointsFetch += tsRange / req.ArchInterval - reqRenderChosenArchive.ValueUint32(uint32(req.Archive)) + if !found { + validIntervalss = append(validIntervalss, validIntervals) + } } - pointsPerSerie := tsRange / interval - if reqs[0].MaxPoints > 0 && pointsPerSerie > reqs[0].MaxPoints { - aggNum := consolidation.AggEvery(pointsPerSerie, reqs[0].MaxPoints) - pointsPerSerie = pointsPerSerie / aggNum - } + // now, we need to pick a combination of intervals from each interval set. + // for each possibly combination of intervals, we compute the LCM interval. + // if if the LCM interval honors maxInterval, we compute the score + // the interval with the highest score wins. + // note : we can probably make this more performant by iterating over + // the retentions, instead of over individual requests + combos := util.AllCombinationsUint32(validIntervalss) + var maxScore int - pointsReturn := uint32(len(reqs)) * pointsPerSerie - reqRenderPointsFetched.ValueUint32(pointsFetch) - reqRenderPointsReturned.ValueUint32(pointsReturn) + lowestInterval := uint32(math.MaxUint32) // lowest interval we find + var candidateInterval uint32 // the candidate MDP-optimized interval + var interval uint32 // will be set to either of the two above + for _, combo := range combos { + candidateInterval = util.Lcm(combo) + if candidateInterval <= maxInterval { + var score int + for _, req := range reqs { + rets := getRetentions(req) + // we know that every request must have a ready retention with an interval that fits into the candidate LCM + // only a matter of finding the best (largest) one + for i := len(rets); i >= 0; i-- { + ret := rets[i] + if uint32(ret.SecondsPerPoint) <= candidateInterval && candidateInterval%uint32(ret.SecondsPerPoint) == 0 && ret.Ready <= from && req.TTL >= minTTL { + score += ret.SecondsPerPoint + } + } + } + if score > maxScore { + maxScore = score + interval = candidateInterval + } + if candidateInterval < lowestInterval { + lowestInterval = candidateInterval + } + } + } + // if we didn't find a suitable MDP-optimized one, just pick the lowest one we've seen. + if interval == 0 { + interval = lowestInterval + } + // now we finally found our optimal interval that we want to use. + // plan all our requests so that they result in the common output interval. + for i := range reqs { + req := &reqs[i] + rets := getRetentions(*req) + for i := len(rets); i >= 0; i-- { + ret := rets[i] + if ret.Ready <= from && req.TTL >= minTTL { + if uint32(ret.SecondsPerPoint) == interval { + req.Plan(i, ret) + break + } + if interval%uint32(ret.SecondsPerPoint) == 0 { + req.Plan(i, ret) + req.PlanNormalization(interval) + break + } + } + } - return reqs, pointsFetch, pointsReturn, nil + } + return reqs, ok } diff --git a/api/query_engine_helpers_test.go b/api/query_engine_helpers_test.go new file mode 100644 index 0000000000..2e0de36480 --- /dev/null +++ b/api/query_engine_helpers_test.go @@ -0,0 +1,38 @@ +package api + +import ( + "github.com/grafana/metrictank/api/models" + "github.com/grafana/metrictank/consolidation" + "github.com/grafana/metrictank/test" +) + +type reqProp struct { + raw uint32 + schemaID uint16 + aggID uint16 +} + +func NewReqProp(raw uint32, schemaID, aggID uint16) reqProp { + return reqProp{raw, schemaID, aggID} +} + +func generate(from, to uint32, reqs []reqProp) ([]models.Req, []models.Req) { + var in []models.Req + mdp := uint32(0) // for these tests, always disable MDP optimization + cons := consolidation.Avg // for these tests, consolidator actually doesn't matter + for i, r := range reqs { + in = append(in, reqRaw(test.GetMKey(i), from, to, mdp, r.raw, cons, r.schemaID, r.aggID)) + } + + out := make([]models.Req, len(in)) + copy(out, in) + return in, out +} + +func adjust(r *models.Req, archive uint8, archInterval, outInterval, ttl uint32) { + r.Archive = archive + r.ArchInterval = archInterval + r.TTL = ttl + r.OutInterval = outInterval + r.AggNum = outInterval / archInterval +} diff --git a/api/query_engine_test.go b/api/query_engine_test.go index 9afd6363d5..5d28a1ea8f 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -1,8 +1,8 @@ package api import ( - "math" "regexp" + "sort" "testing" "github.com/grafana/metrictank/api/models" @@ -12,511 +12,307 @@ import ( "github.com/grafana/metrictank/test" ) -// testAlign verifies the aligment of the given requests, given the retentions (one or more patterns, one or more retentions each) -func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { +func getReqMap(reqs []models.Req) *ReqMap { + rm := NewReqMap() + for _, r := range reqs { + rm.Add(r) + } + return rm +} + +// testPlan verifies the aligment of the given requests, given the retentions (one or more patterns, one or more retentions each) +// passing mpprSoft/mpprHard 0 means we will set them automatically such that they will never be hit +func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, mpprSoft, mpprHard int, t *testing.T) { var schemas []conf.Schema - oriMaxPointsPerReqSoft := maxPointsPerReqSoft + + maxPointsPerReqSoft := mpprSoft for _, ret := range retentions { schemas = append(schemas, conf.Schema{ Pattern: regexp.MustCompile(".*"), Retentions: ret, }) - // make sure maxPointsPerReqSoft is high enough - points := (int(reqs[0].To-reqs[0].From) / ret.Rets[0].SecondsPerPoint) * len(reqs) - if points > maxPointsPerReqSoft { - maxPointsPerReqSoft = points + if mpprSoft == 0 { + // make sure maxPointsPerReqSoft is high enough + points := (int(reqs[0].To-reqs[0].From) / ret.Rets[0].SecondsPerPoint) * len(reqs) + if points > maxPointsPerReqSoft { + maxPointsPerReqSoft = points + } } } + maxPointsPerReqHard = mpprHard + if mpprHard == 0 { + maxPointsPerReqHard = maxPointsPerReqSoft * 10 + } + // Note that conf.Schemas is "expanded" to create a new rule for each rollup + // thus SchemasID must accommodate for this! mdata.Schemas = conf.NewSchemas(schemas) - out, _, _, err := alignRequests(now, reqs[0].From, reqs[0].To, reqs) + //spew.Dump(mdata.Schemas) + out, err := planRequests(now, reqs[0].From, reqs[0].To, getReqMap(reqs), 0, maxPointsPerReqSoft, maxPointsPerReqHard) if err != outErr { t.Errorf("different err value expected: %v, got: %v", outErr, err) } - if len(out) != len(outReqs) { - t.Errorf("different number of requests expected: %v, got: %v", len(outReqs), len(out)) - } else { - for r, exp := range outReqs { - if !exp.Equals(out[r]) { - t.Errorf("request %d:\nexpected: %v\n got: %v", r, exp.DebugString(), out[r].DebugString()) + if err == nil { + if int(out.cnt) != len(outReqs) { + t.Errorf("different number of requests expected: %v, got: %v", len(outReqs), out.cnt) + } else { + got := out.List() + sort.Slice(got, func(i, j int) bool { return test.KeyToInt(got[i].MKey) < test.KeyToInt(got[j].MKey) }) + for r, exp := range outReqs { + if !exp.Equals(got[r]) { + t.Errorf("request %d:\nexpected: %v\n got: %v", r, exp.DebugString(), got[r].DebugString()) + } } } } - - maxPointsPerReqSoft = oriMaxPointsPerReqSoft -} - -// 2 series requested with equal raw intervals. req 0-30. now 1200. one archive of ttl=1200 does it -func TestAlignRequestsBasic(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 60, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 0, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// 2 series requested with equal raw intervals from different schemas. req 0-30. now 1200. their archives of ttl=1200 do it -func TestAlignRequestsBasicDiff(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 60, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// 2 series requested with different raw intervals from different schemas. req 0-30. now 1200. their archives of ttl=1200 do it, but needs normalizing -// (real example seen with alerting queries) -func TestAlignRequestsAlerting(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1200, 0, 0, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 0, 10, 1200, 60, 6), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// 2 series requested with different raw intervals from different schemas. req 0-30. now 1200. neither has long enough archive. no rollups, so best effort from raw -func TestAlignRequestsBasicBestEffort(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 800, 0, 0, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 0, 10, 800, 60, 6), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// 2 series requested with different raw intervals from the same schemas. Both requests should use the 60second rollups -func TestAlignRequestsMultipleIntervalsPerSchema(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(1, 800, 0, 0, 0), - conf.NewRetentionMT(60, 1100, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 60, 800, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// 2 series requested with different raw intervals from the same schemas. Both requests should use raw -func TestAlignRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 30, consolidation.Avg, 0, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 800, 0, 0, 0), - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 0, 10, 800, 30, 3), - reqOut(test.GetMKey(2), 0, 30, 800, 30, consolidation.Avg, 0, 0, 0, 30, 800, 30, 1), - }, - nil, - 800, - t, - ) -} - -// 2 series requested with different raw intervals from different schemas. req 0-30. now 1200. one has short raw. other has short raw + good rollup -func TestAlignRequestsHalfGood(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 800, 0, 0, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), - conf.NewRetentionMT(120, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 0, 10, 800, 120, 12), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 1, 0, 1, 120, 1200, 120, 1), - }, - nil, - 1200, - t, - ) } -// 2 series requested with different raw intervals from different schemas. req 0-30. now 1200. both have short raw + good rollup -func TestAlignRequestsGoodRollup(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1199, 0, 0, 0), // just not long enough - conf.NewRetentionMT(120, 1200, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1199, 0, 0, 0), // just not long enough - conf.NewRetentionMT(120, 1200, 600, 2, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), - }, - nil, - 1200, - t, - ) +// There are a lot of factors to consider. I haven't found a practical way to test all combinations of every factor +// but the approach taken in the functions below should be close enough. +// different test functions: +// * one or both may need to be pushed to rollup to meet TTL +// * use different native resolution within schemas (e.g. skip archives) +// within a test: +// * whether reqs use the exact same schema or different schemas that happen to be identical +// * zero, one or more PNGroups +// * soft limit breach +// * retention ready status +// * whether both need upping interval or not + +func TestPlanRequests_SameInterval_SameTTL_RawOnly_RawMatches(t *testing.T) { + in, out := generate(30, 60, []reqProp{ + NewReqProp(60, 0, 0), + NewReqProp(60, 0, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("60s:1200s:60s:2:true"), + } + adjust(&out[0], 0, 60, 60, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + + // also test what happens when two series use distinct, but equal schemas + rets = append(rets, rets[0]) + in[1].SchemaId, out[1].SchemaId = 1, 1 + testPlan(in, rets, out, nil, 1200, 0, 0, t) + + // also test what happens when one of them hasn't been ready long enough or is not ready at all + for _, r := range []conf.Retentions{ + conf.MustParseRetentions("60s:1200s:60s:2:31"), + conf.MustParseRetentions("60s:1200s:60s:2:false"), + } { + rets[0] = r + //spew.Dump(rets) + //spew.Dump(in) + //spew.Dump(out) + testPlan(in, rets, out, errUnSatisfiable, 1200, 0, 0, t) + } + // but to be clear, when it is ready, it is satisfiable + for _, r := range []conf.Retentions{ + conf.MustParseRetentions("60s:1200s:60s:2:30"), + conf.MustParseRetentions("60s:1200s:60s:2:29"), + conf.MustParseRetentions("60s:1200s:60s:2:true"), + } { + rets[0] = r + testPlan(in, rets, out, nil, 1200, 0, 0, t) + } } -// 2 series requested with different raw intervals, and rollup intervals from different schemas. req 0-30. now 1200. both have short raw + good rollup -func TestAlignRequestsDiffGoodRollup(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1199, 0, 0, 0), // just not long enough - conf.NewRetentionMT(100, 1200, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1199, 0, 0, 0), // just not long enough - conf.NewRetentionMT(600, 1200, 600, 2, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 100, 1200, 600, 6), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0, 1, 600, 1200, 600, 1), - }, - nil, - 1200, - t, - ) +/* +func copy2(a, b []models.Req) ([]models.Req, []models.Req) { + a2 := make([]models.Req, len(a)) + b2 := make([]models.Req, len(b)) + copy(a2, a) + copy(b2, b) + return a2, b2 } +*/ -// now raw is short and we have a rollup we can use instead, at same interval as one of the raws -func TestAlignRequestsWeird(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1199, 0, 0, 0), - conf.NewRetentionMT(60, 1200, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// now TTL of first rollup is *just* enough -func TestAlignRequestsWeird2(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1100, 0, 0, 0), // just not long enough - conf.NewRetentionMT(120, 1200, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), // just not long enough - conf.NewRetentionMT(120, 1200, 600, 2, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), - }, - nil, - 1200, - t, - ) -} +func TestPlanRequests_DifferentInterval_SameTTL_RawOnly_RawMatches(t *testing.T) { + in, out := generate(0, 30, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(60, 1, 0), + }) + adjust(&out[0], 0, 10, 10, 1200) + adjust(&out[1], 0, 60, 60, 1200) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:1200s:60s:2:true"), + conf.MustParseRetentions("60s:1200s:60s:2:true"), + } + t.Run("NoPNGroups", func(t *testing.T) { + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) -// now TTL of first rollup is not enough but we have no other choice but to use it -func TestAlignRequestsNoOtherChoice(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1100, 0, 0, 0), - conf.NewRetentionMT(120, 1199, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), - conf.NewRetentionMT(120, 1199, 600, 2, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 120, 1199, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 2, 0, 1, 120, 1199, 120, 1), - }, - nil, - 1200, - t, - ) + t.Run("DifferentPNGroups", func(t *testing.T) { + // nothing should change + in[0].PNGroup, out[0].PNGroup = 123, 123 + in[1].PNGroup, out[1].PNGroup = 124, 124 + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) + t.Run("SamePNGroups", func(t *testing.T) { + // should be normalized to the same interval + in[0].PNGroup, out[0].PNGroup = 123, 123 + in[1].PNGroup, out[1].PNGroup = 123, 123 + adjust(&out[0], 0, 10, 60, 1200) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) } -// now TTL of first rollup is not enough and we have a 3rd band to use -func TestAlignRequests3rdBand(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(1, 1100, 0, 0, 0), - conf.NewRetentionMT(120, 1199, 600, 2, 0), - conf.NewRetentionMT(240, 1200, 600, 2, 0), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), - conf.NewRetentionMT(240, 1200, 600, 2, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 2, 240, 1200, 240, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0, 1, 240, 1200, 240, 1), - }, - nil, - 1200, - t, - ) -} +func TestPlanRequests_DifferentInterval_DifferentTTL_RawOnly_1RawShort(t *testing.T) { + in, out := generate(0, 1000, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(60, 1, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:800s:60s:2:true"), + conf.MustParseRetentions("60s:1080s:60s:2:true"), + } + adjust(&out[0], 0, 10, 10, 800) + adjust(&out[1], 0, 60, 60, 1080) + t.Run("NoPNGroups", func(t *testing.T) { + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) -// now TTL of raw/first rollup is not enough but the two rollups are disabled, so must use raw -func TestAlignRequests2RollupsDisabled(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(10, 1100, 0, 0, 0), // just not long enough - conf.NewRetentionMT(120, 1199, 600, 2, math.MaxUint32), - conf.NewRetentionMT(240, 1200, 600, 2, math.MaxUint32), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), // just not long enough - conf.NewRetentionMT(240, 1200, 600, 2, math.MaxUint32), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 0, 10, 1100, 60, 6), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - t, - ) -} -func TestAlignRequestsHuh(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(1, 1000, 0, 0, 0), - conf.NewRetentionMT(120, 1080, 600, 2, 0), - conf.NewRetentionMT(240, 1200, 600, 2, math.MaxUint32), - ), - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1100, 0, 0, 0), - conf.NewRetentionMT(240, 1200, 600, 2, math.MaxUint32), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 800, 10, consolidation.Avg, 0, 0, 1, 120, 1080, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 800, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 120, 2), - }, - nil, - 1200, - t, - ) -} + t.Run("DifferentPNGroups", func(t *testing.T) { + // nothing should change + in[0].PNGroup, out[0].PNGroup = 123, 123 + in[1].PNGroup, out[1].PNGroup = 124, 124 + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) + t.Run("SamePNGroups", func(t *testing.T) { + // should be normalized to the same interval + in[0].PNGroup, out[0].PNGroup = 123, 123 + in[1].PNGroup, out[1].PNGroup = 123, 123 + adjust(&out[0], 0, 10, 60, 800) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) -func TestAlignRequestsDifferentReadyStates(t *testing.T) { - testAlign([]models.Req{ - reqRaw(test.GetMKey(1), 100, 300, 800, 1, consolidation.Avg, 0, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(1, 300, 120, 5, 0), // TTL not good enough - conf.NewRetentionMT(5, 450, 600, 4, math.MaxUint32), // TTL good, but not ready - conf.NewRetentionMT(10, 460, 600, 3, 150), // TTL good, but not ready since long enough - conf.NewRetentionMT(20, 470, 600, 2, 101), // TTL good, but not ready since long enough - conf.NewRetentionMT(60, 480, 600, 1, 100), // TTL good and ready since long enough - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 100, 300, 800, 1, consolidation.Avg, 0, 0, 4, 60, 480, 60, 1), - }, - nil, - 500, - t, - ) } -var hour uint32 = 60 * 60 -var day uint32 = 24 * hour +func TestPlanRequests_DifferentInterval_DifferentTTL_1RawOnly1RawAndRollups_1Raw1Rollup(t *testing.T) { + in, out := generate(0, 1000, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(60, 2, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:1080s:60s:2:true,30s:1500s:60s:2:true"), + conf.MustParseRetentions("60s:1320s:60s:2:true"), + } + adjust(&out[0], 1, 30, 30, 1500) + adjust(&out[1], 0, 60, 60, 1320) + t.Run("Base", func(t *testing.T) { + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) -func testMaxPointsPerReq(maxPointsSoft, maxPointsHard int, reqs []models.Req, t *testing.T) ([]models.Req, error) { - origMaxPointsPerReqSoft := maxPointsPerReqSoft - origMaxPointsPerReqHard := maxPointsPerReqHard - maxPointsPerReqSoft = maxPointsSoft - maxPointsPerReqHard = maxPointsHard + t.Run("SameButTTLsNotLongEnough", func(t *testing.T) { + rets = []conf.Retentions{ + conf.MustParseRetentions("10s:1080s:60s:2:true,30s:1140s:60s:2:true"), + conf.MustParseRetentions("60s:1020s:60s:2:true"), + } + adjust(&out[0], 1, 30, 30, 1140) + adjust(&out[1], 0, 60, 60, 1020) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) - mdata.Schemas = conf.NewSchemas([]conf.Schema{{ - Pattern: regexp.MustCompile(".*"), - Retentions: conf.BuildFromRetentions( - conf.NewRetentionMT(1, 2*day, 600, 2, 0), - conf.NewRetentionMT(60, 7*day, 600, 2, 0), - conf.NewRetentionMT(3600, 30*day, 600, 2, 0), - ), - }}) + t.Run("ArchiveWeNeedIsNotReady", func(t *testing.T) { + rets[0] = conf.MustParseRetentions("10s:1080s:60s:2:true,30s:1500s:60s:2:false") + rets[1] = conf.MustParseRetentions("60s:1320s:60s:2:true") + adjust(&out[0], 0, 10, 10, 1080) + adjust(&out[1], 0, 60, 60, 1320) + //spew.Dump(rets) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + }) - out, _, _, err := alignRequests(30*day, reqs[0].From, reqs[0].To, reqs) - maxPointsPerReqSoft = origMaxPointsPerReqSoft - maxPointsPerReqHard = origMaxPointsPerReqHard - return out, err } -func TestGettingOneNextBiggerAgg(t *testing.T) { - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 30*day, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), - } - - // without maxPointsPerReqSoft = 23*hour we'd get archive 0 for this request, - // with it we expect the aggregation to get bumped to the next one - out, err := testMaxPointsPerReq(int(23*hour), 0, reqs, t) - if err != nil { - t.Fatalf("expected to get no error") - } - if out[0].Archive != 1 { - t.Errorf("expected archive %d, but got archive %d", 1, out[0].Archive) +// like the above test, except the one that was already long enough has a rollup (that we don't use) +// and the short one only has raw. +func TestPlanRequests_DifferentInt_DifferentTTL_1RawOnly1RawAndRollups_1RawShort(t *testing.T) { + in, out := generate(0, 1000, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(60, 1, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:800s:60s:2:true"), + conf.MustParseRetentions("60s:1200s:60s:2:true,5m:3000s:5min:2:true"), // extra rollup that we don't care for } + adjust(&out[0], 0, 10, 10, 800) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, 0, 0, t) + + t.Run("RawArchiveNotReady", func(t *testing.T) { + // should switch to rollup + rets[1] = conf.MustParseRetentions("60s:1200s:60s:2:false,5m:3000s:5min:2:true") + adjust(&out[1], 1, 300, 300, 3000) + testPlan(in, rets, out, nil, 3000, 0, 0, t) + }) } -func TestGettingTwoNextBiggerAgg(t *testing.T) { - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 30*day, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), - } - - // maxPointsPerReqSoft only allows 24 points, so the aggregation 2 with - // 3600 SecondsPerPoint should be chosen for our request of 1 day - out, err := testMaxPointsPerReq(24, 0, reqs, t) - if err != nil { - t.Fatalf("expected to get no error") - } - if out[0].Archive != 2 { - t.Errorf("expected archive %d, but got archive %d", 2, out[0].Archive) +// 2 series with different raw intervals from the same schemas. Both requests should use the raw archive +func TestPlanRequestsMultiIntervalsUseRaw(t *testing.T) { + in, out := generate(0, 1000, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(30, 1, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:800s:60s:2:true,60s:1200s:5min:2:true"), } + adjust(&out[0], 0, 10, 10, 800) + adjust(&out[1], 0, 30, 30, 1200) + testPlan(in, rets, out, nil, 800, 0, 0, t) + + // let's look at some MaxPointsPerReqSoft scenarios: + // points fetched is for each request (to-from) / archInterval + // so normally, 1000/10 + 1000 / 30 = ~133 + t.Run("WithMaxPointsPerReqSoftVeryTight", func(t *testing.T) { + // this should still work as before, but just make the limit + testPlan(in, rets, out, nil, 800, 134, 0, t) + }) + t.Run("WithMaxPointsPerReqSoftBreached", func(t *testing.T) { + // we breach so, one request at a time, it'll lower the resolution, if an interval is available... + adjust(&out[0], 1, 60, 60, 1200) + adjust(&out[1], 0, 30, 30, 1200) + testPlan(in, rets, out, nil, 800, 130, 0, t) + t.Run("WithHardVeryTight", func(t *testing.T) { + // 1000/60 + 1000/30 =~ 46 + adjust(&out[0], 1, 60, 60, 1200) + adjust(&out[1], 0, 30, 30, 1200) + testPlan(in, rets, out, nil, 800, 130, 50, t) + }) + t.Run("WithHardBreached", func(t *testing.T) { + // 1000/60 + 1000/30 =~ 46 + adjust(&out[0], 1, 60, 60, 1200) + adjust(&out[1], 0, 30, 30, 1200) + testPlan(in, rets, out, errMaxPointsPerReq, 800, 130, 40, t) + }) + }) } -func TestMaxPointsPerReqHardLimit(t *testing.T) { - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 30*day, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), - } - // we're requesting one day and the lowest resolution aggregation has 3600 seconds per point, - // so there should be an error because we only allow max 23 points per request - _, err := testMaxPointsPerReq(22, 23, reqs, t) - if err != errMaxPointsPerReq { - t.Fatalf("expected to get an error") +// 3 series with different raw intervals from the same schemas. TTL causes both to go to first rollup, which for one of them is raw +func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { + in, out := generate(0, 1000, []reqProp{ + NewReqProp(1, 0, 0), + NewReqProp(10, 1, 0), + NewReqProp(60, 1, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("1s:800s:2h:2:true,60s:1140s:1h:2:true"), } + adjust(&out[0], 1, 60, 60, 1140) + adjust(&out[1], 0, 10, 10, 1140) // note how it has archive 10 + adjust(&out[2], 0, 60, 60, 1140) + testPlan(in, rets, out, nil, 1200, 0, 0, t) } -var result []models.Req +var result *ReqsPlan -func BenchmarkAlignRequests(b *testing.B) { - var res []models.Req - reqs := []models.Req{ - reqRaw(test.GetMKey(1), 0, 3600*24*7, 1000, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 3600*24*7, 1000, 30, consolidation.Avg, 4, 0), - reqRaw(test.GetMKey(3), 0, 3600*24*7, 1000, 60, consolidation.Avg, 8, 0), - } +func BenchmarkPlanRequestsSamePNGroupNoLimits(b *testing.B) { + var res *ReqsPlan + reqs := NewReqMap() + reqs.Add(reqRaw(test.GetMKey(1), 0, 3600*24*7, 0, 10, consolidation.Avg, 0, 0)) + reqs.Add(reqRaw(test.GetMKey(2), 0, 3600*24*7, 0, 30, consolidation.Avg, 4, 0)) + reqs.Add(reqRaw(test.GetMKey(3), 0, 3600*24*7, 0, 60, consolidation.Avg, 8, 0)) mdata.Schemas = conf.NewSchemas([]conf.Schema{ { Pattern: regexp.MustCompile("a"), @@ -548,7 +344,7 @@ func BenchmarkAlignRequests(b *testing.B) { }) for n := 0; n < b.N; n++ { - res, _, _, _ = alignRequests(14*24*3600, 0, 3600*24*7, reqs) + res, _ = planRequests(14*24*3600, 0, 3600*24*7, reqs, 0, 0, 0) } result = res } diff --git a/cmd/mt-explain/main.go b/cmd/mt-explain/main.go index f3da607fa6..c8f8fc96f7 100644 --- a/cmd/mt-explain/main.go +++ b/cmd/mt-explain/main.go @@ -25,6 +25,9 @@ func main() { to := flag.String("to", "now", "get data until (exclusive)") mdp := flag.Int("mdp", 800, "max data points to return") timeZoneStr := flag.String("time-zone", "local", "time-zone to use for interpreting from/to when needed. (check your config)") + var optimizations expr.Optimizations + flag.BoolVar(&optimizations.PreNormalization, "pre-normalization", true, "enable pre-normalization optimization") + flag.BoolVar(&optimizations.MDP, "mdp-optimization", false, "enable MaxDataPoints optimization (experimental)") flag.Usage = func() { fmt.Println("mt-explain") @@ -32,6 +35,7 @@ func main() { fmt.Println() fmt.Printf("Usage:\n\n") fmt.Printf(" mt-explain\n") + flag.PrintDefaults() fmt.Println() fmt.Printf("Example:\n\n") fmt.Printf(" mt-explain -from -24h -to now -mdp 1000 \"movingAverage(sumSeries(foo.bar), '2min')\" \"alias(averageSeries(foo.*), 'foo-avg')\"\n\n") @@ -76,7 +80,7 @@ func main() { return } - plan, err := expr.NewPlan(exps, fromUnix, toUnix, uint32(*mdp), *stable, nil) + plan, err := expr.NewPlan(exps, fromUnix, toUnix, uint32(*mdp), *stable, optimizations) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { fmt.Printf("Unsupported function %q: must defer query to graphite\n", string(fun)) diff --git a/devdocs/expr.md b/devdocs/expr.md index ef0b842633..61b36ab3fb 100644 --- a/devdocs/expr.md +++ b/devdocs/expr.md @@ -1,4 +1,37 @@ -## Management of point slices +# Considerations when writing a Graphite processing function in metrictank + +## constructors should return a pointer + +* so that properties can be set +* we get a consistent PNGroup through the pipeline (if applicable) + +(the exception here is the data loading function FuncGet() which doesn't need to set any properties) + +## consider whether the function is GR, IA, a transparent or opaque aggregation, or combine different series together somehow + +Such functions require special options. +see https://github.com/grafana/metrictank/issues/926#issuecomment-559596384 + +## implement our copy-on-write approach when dealing with modifying series + +See section 'Considerations around Series changes and reuse and why we chose copy-on-write' below. + +* must not modify existing data that the fields of any pre-existing `models.Series` point to. At the time of writing it's only the Datapoints, Tags and Meta fields, but this may change. + (exception: FuncGet) +* should use the pool to get new slices in which to store their new/modified datapoints. +* should add said new slices into the cache so that when the plan has run, and the caller calls plan.Clean(), we can return its datapoints slice to the pool. +* the other purpose of the cache is to add processed data to the set of available data such that other functions could reuse it, but this mechanism is not implemented yet. + That's why we always add to the cache without bothering to set the right request key (`cache[Req{}]`). + +example: an averageSeries() of 3 series: +* will create an output series value. +* it will use a new datapoints slice, retrieved from pool, because the points will be different. also it will allocate a new meta section and tags map because they are different from the input series also. +* won't put the 3 inputs back in the pool or cache, because whoever allocated the input series was responsible for doing that. we should not add the same arrays to the pool multiple times. +* It will however store the newly created series into the cache such that during plan cleanup time, the series' datapoints slice will be moved back to the pool. + +# Considerations around Series changes and reuse and why we chose copy-on-write. + +## introduction The `models.Series` type, even when passed by value, has a few fields that need special attention: * `Datapoints []schema.Point` @@ -8,12 +41,15 @@ The `models.Series` type, even when passed by value, has a few fields that need Many processing functions will want to return an output series that differs from the input, in terms of (some of the) datapoints may have changed value, tags or metadata. They need a place to store their output but we cannot simply operate on the input series, or even a copy of it, as the underlying datastructures are shared. -Goals: +## Goals + * processing functions should not modify data if that data needs to remain original (e.g. because of re-use of the same input data elsewhere) * minimize allocations of new structures foremost * minimize data copying as a smaller concern * simple code +# Implementation + there's 2 main choices: 1) copy-on-write: @@ -32,11 +68,11 @@ there's 2 main choices: - means we cannot cache intermediate results, unless we also make deep copies anny time we want to cache and hand off for further processing. -for now we assume that multi-steps in a row is not that common, and COW seems more commonly the best approach, so we chose COW. +for now we assume that multi-steps in a row is not that common, and COW seems more commonly the best approach, so we chose **the copy on write approach** This leaves the problem of effectively managing allocations and using a sync.Pool. -Note that the expr library can be called by different clients (MT, grafana, graphite-ng, ...) +Note that the expr library can be called by different clients. At this point only Metrictank uses it, but we intend this library to be easily embeddable in other programs. It's up to the client to instantiate the pool, and set up the default allocation to return point slices of desired point capacity. The client can then of course use this pool to store series, which it then feeds to expr. expr library does the rest. It manages the series/pointslices and gets new ones as a basis for the COW. @@ -44,19 +80,7 @@ Once the data is returned to the client, and the client is done using the return which returns all data back to the pool (both input data or newly generated series, whether they made it into the final output or not). -function implementations: - -* must not modify existing slices or maps or other composite datastructures (at the time of writing, it's only slices/maps), with the exception of FuncGet. -* should use the pool to get new slices in which to store their new/modified datapoints. -* should add said new slices into the cache so it can later be cleaned - -example: an averageSeries() of 3 series: -* will create an output series value. -* it will use a new datapoints slice, retrieved from pool, because the points will be different. also it will allocate a new meta section and tags map because they are different from the input series also. -* won't put the 3 inputs back in the pool, because whoever allocated the input series was responsible for doing that. we should not add the same arrays to the pool multiple times. -* It will however store the newly created series into the pool such that it can later be reclaimed. - -## consolidateBy +# consolidateBy consolidateBy(, "fun") defines a consolidation function to be applied to "". there are some subtle behaviors we have to cater to, and we do it like so: @@ -100,7 +124,7 @@ So: - consolidateBy setting defined closest to the leaf without a special* function in between the setting and the leaf, if available - determined via storage-aggregation.conf (defaults to average) 3) at execution time, the consolidation settings encountered in consolidateBy calls travel up to the root because it is configured on the series, which is passed through the various layers of processing until it hits the root and the output step. This becomes useful in two cases: - - when series need to be normalized at runtime, e.g. for sumSeries or divideSeries with series that have different steps; they need to be normalized (consolidated) so that the series get a compatible step, and the default of "avg" may not suffice. (note that right now we have alignRequests which normalizes all series at fetch time, which can actually be a bit too eager, because some requests can use multiple targets with different processing - e.g. feed two different series into summarize(), so we actually don't need to normalize at runtime, but in the future we should make this better - TODO) + - when series need to be normalized at runtime, e.g. for sumSeries or divideSeries with series that have different steps; they need to be normalized (consolidated) so that the series get a compatible step, and the default of "avg" may not suffice. (note that right now we have alignRequests which normalizes all series at fetch time, which can actually be a bit too eager, because some requests can use multiple targets with different processing - e.g. feed two different series into summarize(), so we actually don't need to normalize at runtime, but in the future we should make this better - TODO THIS IS OUT OF DATE) - when returning data back to the user via a json response and whatnot, we can consolidate down using the method requested by the user (or average, if not specified). Likewise here, when the setting encounters a special* function while traveling up to the root, the consolidation value is reset to the default (average) Note: some functions combine multiple series into a new one (e.g. sumSeries, avgSeries, ...). Your input series may use different consolidateBy settings, some may be explicitly specified while others are not. In this scenario, the output series will be given the first explicitly defined consolidateBy found by iterating the inputs, or the first default otherwise. @@ -113,7 +137,7 @@ see also https://github.com/grafana/metrictank/issues/463#issuecomment-275199880 -## naming +# naming when requesting series directly, we want to show the target (to uniquely identify each series), not the queryPattern thus the output returned by plan.Run must always use Target attribute to show the proper name diff --git a/devdocs/maxdatapoints.txt b/devdocs/maxdatapoints.txt new file mode 100644 index 0000000000..a3997733d6 --- /dev/null +++ b/devdocs/maxdatapoints.txt @@ -0,0 +1,24 @@ +# master + +mdp set from GET param, but 0 if came from graphite + -> NewPlan() + -> plan.MaxDatapoints + -> executePlan() -> models.NewReq() + -> used in alignRequests() to report on number of points that will be returned (only looks at 1 request, assumes all are same) + -> plan.MaxDatapoints also used for final runtime consolidation + +# our branch + +main change is that Context and expr.Req now have MDP fields, to control (only) fetching (MDP-optimization), +whereas plan.MaxDataPoints is still the one controlling runtime consolidation and reporting on the number of points in planRequests(). + +mdp set from GET param, but 0 if came from graphite + -> NewPlan() + -> plan.MaxDatapoints + -> Context.MDP, though GR functions like (smart)Summarize set r.MDP =0 + -> expr.NewReqFromContext() + -> expr.Req.MDP + -> executePlan() models.NewReq() -> models.Req.MaxPoints + -> planRequests(): used for MDP-optimization + -> plan.MaxDatapoints used for final runtime consolidation + -> and also used in planRequests() for reporting diff --git a/devdocs/render-request-handling.md b/devdocs/render-request-handling.md index a694491973..75acc139cb 100644 --- a/devdocs/render-request-handling.md +++ b/devdocs/render-request-handling.md @@ -12,9 +12,29 @@ * finds all series by fanning out the query patterns to all other shards. this gives basically idx.Node back. has the path, leaf, metricdefinition, schema/aggregation(rollup) settings, for each series, as well as on which node it can be found. * construct models.Req objects for each serie. this uses the MKey to identify series, also sets from/to, maxdatapoints, etc. - * `alignRequests`: this looks at all models.Req objects and aligns them to a common step. - it selects the archive to use, consolidator settings etc (see NOTES in expr directory for more info) + * `planRequests`: this plans all models.Req objects, deciding which archive to read from, whether to apply normalization, etc. + (see NOTES in expr directory for more info) * `getTargets`: gets the data from the local node and peer nodes based on the models.Req objects * `mergeSeries`: if there's multiple series with same name/tags, from, to and consolidator (e.g. because there's multiple series because users switched intervals), merge them together into one series * Sort each merged series so that the output of a function is well-defined and repeatable. * `plan.Run`: invoke all function processing, followed by runtime consolidation as necessary + +## MDP-optimization + +MDP at the leaf of the expr tree (fetch request) of 0 means don't optimize. If set it to >0 it means the request can be optimized. +When the data may be subjected to a GR-function, we set it to 0. +How do we achieve this? +* MDP at the root is set 0 if the request came from graphite or to MaxDataPoints otherwise. +* as the context flows from root through the processing functions to the data requests, if we hit a GR function, we set MDP to 0 on the context (and thus also on any subsequent requests) + +## Pre-normalization + +Any data requested (checked at the leaf node of the expr tree) should have its own independent interval. +However, multiple series getting fetched that then get aggregated together may be pre-normalized if they are part of the same pre-normalization-group (have a common PNGroup that is > 0). +(for more details see devdocs/alignrequests-too-course-grained.txt) +The mechanics here are: +* we set PNGroup to 0 by default on the context, which gets inherited down the tree +* as we traverse down tree: transparent aggregations set PNGroups to the pointer value of that function, to uniquely identify any further data requests that will be fed into the same transparent aggregation. +* as we traverse down, any opaque aggregation functions and IA-functions reset PNGroup back to 0. Note that currently all known IA functions are also GR functions and vice versa. Meaning, + as we pass functions like smartSummarize which should undo MDP-optimization, they also undo pre-normalization. + diff --git a/docker/docker-chaos/metrictank.ini b/docker/docker-chaos/metrictank.ini index 06c2026c12..959629e395 100644 --- a/docker/docker-chaos/metrictank.ini +++ b/docker/docker-chaos/metrictank.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/docker/docker-cluster-query/metrictank.ini b/docker/docker-cluster-query/metrictank.ini index f77a8e226c..929c62a2d5 100644 --- a/docker/docker-cluster-query/metrictank.ini +++ b/docker/docker-cluster-query/metrictank.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/docker/docker-cluster/metrictank.ini b/docker/docker-cluster/metrictank.ini index 33e0526d36..eb6bfdb640 100644 --- a/docker/docker-cluster/metrictank.ini +++ b/docker/docker-cluster/metrictank.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/docker/docker-dev-custom-cfg-kafka/metrictank.ini b/docker/docker-dev-custom-cfg-kafka/metrictank.ini index fb1e2b7674..1d5c0cac54 100644 --- a/docker/docker-dev-custom-cfg-kafka/metrictank.ini +++ b/docker/docker-dev-custom-cfg-kafka/metrictank.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/docs/config.md b/docs/config.md index e95fbce6e6..5e17b14114 100644 --- a/docs/config.md +++ b/docs/config.md @@ -293,6 +293,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ``` ## metric data inputs ## diff --git a/docs/consolidation.md b/docs/consolidation.md index a639b7fa87..10dd6d092d 100644 --- a/docs/consolidation.md +++ b/docs/consolidation.md @@ -36,7 +36,7 @@ This further reduces data at runtime on an as-needed basis. It supports min, max, sum, average. -## The request alignment algorithm +## The request planning algorithm. OUT OF DATE AS OF https://github.com/grafana/metrictank/pull/951 Metrictank uses a function called `alignRequests` which will: diff --git a/docs/http-api.md b/docs/http-api.md index adc1310e3e..32108d6a86 100644 --- a/docs/http-api.md +++ b/docs/http-api.md @@ -186,6 +186,7 @@ POST /render - none: always defer to graphite for processing. If metrictank doesn't have a requested function, it always proxies to graphite, irrespective of this setting. +* optimizations: can override http.pre-normalization and http.mdp-optimization options. empty (default) : no override. either "none" to force no optimizations, or a csv list with either of both of "pn", "mdp" to enable those options. Data queried for must be stored under the given org or be public data (see [multi-tenancy](https://github.com/grafana/metrictank/blob/master/docs/multi-tenancy.md)) @@ -234,7 +235,7 @@ Each lineage section has these fields: | schema-retentions | Retentions defined in storage-schemas.conf | | archive-read | Which archive was read as defined in the retentions. (0 means raw, 1 first rollup, etc) | | archive-interval | The native interval of the archive that was read | -| aggnum-norm | If >1, number of points aggregated together per point, as part of normalization (series alignment) | +| aggnum-norm | If >1, number of points aggregated together per point, as part of normalization | | aggnum-rc | If >1, number of points aggregated together per output point, as part of runtime consolidation (MaxDataPoints) | | consolidator-normfetch | Consolidator used for normalization (if aggnum-norm > 1) and which rollup was read (if archive-read > 0) | | consolidator-rc | Consolidator used for runtime consolidation (MaxDataPoints) (if aggnum-rc > 1) | diff --git a/docs/metrics.md b/docs/metrics.md index c36805f3ff..5c9d77fe86 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -24,7 +24,8 @@ the archive chosen for the request. * `api.request.render.points_fetched`: the number of points that need to be fetched for a /render request. * `api.request.render.points_returned`: -the number of points the request will return. +the number of points the request will return +best effort: not aware of summarize(), aggregation functions, runtime normalization. but does account for runtime consolidation * `api.request.render.series`: the number of series a /render request is handling. This is the number of metrics after all of the targets in the request have expanded by searching the index. diff --git a/docs/render-path.md b/docs/render-path.md index bb7b68f326..58d53c2c9e 100644 --- a/docs/render-path.md +++ b/docs/render-path.md @@ -14,7 +14,7 @@ is quantized to a series with points with timestamps 60, 70, 80, 90 (null), 100. a series is fixed, with respect to a query with from/to time, when * it is quantized. -* contains only timestamps such that `from <= timestamps < to`. +* contains all the timestamps such that `from <= timestamps < to`, but no more. ## canonical form @@ -25,11 +25,11 @@ with respect to how many points it contains and which timestamps they have. It is important here to keep in mind that consolidated points get the timestamp of the last of its input points. Continuing the above example, if we need to normalize the above series with aggNum 3 (OutInterval is 30s) -we would normally get a series of (60,70,80), (90, 100, 110 - null), so the 30-second timestamps become 80 and 110. +we would normally get a series of (60,70,80), (90 -null -, 100, 110 - null), so the 30-second timestamps become 80 and 110. But this is not the quantized form of a series with an interval of 30. So, what typically happens to make a series canonical, is at fetch time, also fetch some extra earlier data. -such that after aggregation, the series looks like (40, 50, 60) (70, 80, 90) (100, 110 -no data, 120 - no data) or 60, 90, 120. +such that after aggregation, the series looks like (40, 50, 60) (70, 80, 90) (100, 110 -null -, 120 - null -) which results in timestamps 60, 90, 120. Technically speaking we don't have to fetch the earlier points, we could leave them null, but then the data would be misleading. It's better for the first aggregate point to have received its full input set of raw data points. @@ -56,6 +56,12 @@ Continuing the example again, it could be another series that had a raw interval a pre-canonical series is simply a series that after normalizing, will be canonical. I.O.W. is a series that is fetched in such a way that when it is fed to Consolidate(), will produce a canonical series. See above for more details. +Note: this can only be done to the extent we know what the normalization looks like. +(by setting up req.AggNum and req.OutInterval for normalization). For series that get (further) normalized at runtime, +we can't predict this at fetch time and have to remove points to make the output canonical, or do what Graphite also does, +which is to add null points at the beginning or end as needed, which may lead to inaccurate leading or trailing points that +go potentially out of the bounds of the query. + ## nudging @@ -84,11 +90,112 @@ given multiple series being fetched of different resolution, normalizing is runt but only for the purpose of bringing series of different resolutions to a common, lower resolution such that they can be used together (for aggregating, merging, etc) +# Optimizations + +Metrictank has two specific optimizations that can be enabled with the config settings: + +``` +[http] +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false +``` + +We explain them in detail below. + +## Pre-normalization + +First, let's look at some definitions. + +### Interval-altering function + +Certain functions will return output series in an interval different from the input interval. +For example summarize() and smartSummarize(). We refer to these as IA-functions below. +In principle we can predict what the output interval will be during the plan phase, because we can parse the function arguments. +However, for simplicity, we don't implement this and treat all IA functions as functions that may change the interval of series in unpredicatable ways. + +### Transparent aggregation + +A transparent aggregation is a processing function that aggregates multiple series together in a predictable way (meaning: known at planning time, before fetching the data). +E.g. sumSeries, averageSeries are known to always aggregate all their inputs together. + +### Opaque aggregation + +An opaque aggregation is a processing function where we cannot accurately predict which series will be aggregated together +because it depends on information (e.g. names, tags) that will only be known at runtime. (e.g. groupByTags, groupByNode(s)) + +### Pre-normalizable + +In the past, metrictank used to always align all series to the same resolution. But that was a limitation and we don't do this anymore (#926). +Generally, if series have different intervals, they can keep those and we return them in whichever interval works best for them. + +However, when data will be used together (e.g. aggregating multiple series together, or certain functions like divideSeries, asPercent, etc) they will need to have the same interval. +An aggregation can be opaque or transparent as defined above. + +Pre-normalizing is when we can safely - during planning - set up normalization to happen right after fetching (or better: set up the fetch parameters such that normalizing is not needed) and when we know the normalization won't affect anything else. + +This is the case when series go from fetching to a processing function like: +* a transparent aggregation +* asPercent in a certain mode (where it has to normalize all inputs) + +possibly with some processing functions in between the fetching and the above function, except opaque aggregation(s) or IA-function(s). + +Some functions also have to normalize (some of) their inputs, but yet cannot have their inputs pre-normalized. For example, +divideSeries because it applies the same divisor to multiple distinct dividend inputs (of possibly different intervals). + +For example if we have these schemas: +``` +series A: 1s:1d,10s:1y +series B: 10s:1d +``` + +Let's say the initial fetch parameters are to get the raw data for both A and B. +If we know that these series will be aggregated together, they will need to be normalized, meaning A will need to be at 10s resolution. +If the query is `sum(A,B)` or `sum(perSecond(A),B)` we can safely pre-normalize, specifically, we can fetch the first rollup of series A, rather than fetching the raw data +and then normalizing (consolidating) at runtime - and thus spend less resources - because we know for sure that having the coarser data for A will not cause trouble in this pipeline. +However, if the query is `sum(A, summarize(B,...))` we cannot safely do this as we don't have a prediction of what the output interval of `summarize(B,...)` will be. +Likewise, if the query is `groupByNode(group(A,B), 2, callback='sum')` we cannot predict whether A and B will end up in the same group, and thus should be normalized. + +Benefits of this optimization: +1) less work spent consolidating at runtime, less data to fetch +2) it assures data will be fetched in a pre-canonical way. If we don't set up normalization for fetching, data may not be pre-canonical, which means we may have to add null points to normalize it to canonical data, lowering the accuracy of the first or last point. +3) pre-normalized data reduces a request's chance of breaching max-points-per-req-soft and thus makes it less likely that other data that should be high-resolution gets fetched in a coarser way. +when it eventually needs to be normalized at runtime, points at the beginning or end of the series may be less accurate. + +Downsides of this optimization: +1) if you already have the raw data cached, and the rollup data is not cached yet, it may result in a slower query, and you'd use slightly more chunk cache after the fetch. But this is an edge case + +## MDP-optimizable + +### Greedy-resolution functions + +A Greedy-resolution function (GR-function) is a certain processing function that requires, or may require, high resolution data input to do their computations, even if their output will be consolidated down (due to maxDatapoints setting) +For example summarize(). +For these, we should return as high-resolution data as we can. + +### MDP-optimizable + +MDP-optimizable aka maxDataPoints-optimizable is a data request where we can safely fetch lower precision data by taking into account MaxDataPoints-based consolidation that will take place after function processing. +A data request is currently considered MDP-optimizable if we know for sure that it won't be subjected to GR-functions. +I.O.W. when both of these conditions are true: +* the client was an end-user, not Graphite (Graphite may run any processing, such as GR-functions, without telling us) +* we (metrictank) will not run GR-functions on this data + +What kind of optimizations can we do? Consider this retention rule: + +`1s:1d,10s:1y` +request from=now-2hours to=now, MDP=800 +Our options are: +* 7200 raw (archive 0) datapoints, consolidate aggNum 9, down to 800 (by the way, current code does generate "odd" intervals like 9s in this case) +* 720 datapoints of archive 1. -## request flow +While archive 1 is a bit less accurate, it is less data to load, decode, and requires no consolidation. We have a strong suspicion that it is less costly to use this data to satisfy the request. +(a bit of history here: in the early days, we used to always apply basically this optimization to *all* requests. This turned out to be a bad idea when we realized what happened with GR-functions. In #463 we decided to simply disable all optimizations and always fetch raw data for everything. This assured correctness, but also was needlessly aggressive for certain types of requests discussed here.) -TODO talk about -alignRequests -> getTargets -> mergeSeries -> sort Series -> plan.Run (executes functions and does MDP consolidation with nudging) +However, there are a few concerns not fully fleshed out. +* Targeting a number of points of MDP/2 seems fine for typical charts with an MDP of hundreds or thousands of points. Once people request values like MDP 1, 2 or 3 it becomes icky. +* For certain queries like `avg(consolidateBy(seriesByTags(...), 'max'))` or `seriesByTag('name=requests.count') | consolidateBy('sum') | scaleToSeconds(1) | consolidateBy('max')`, that have different consolidators for normalization and runtime consolidation, would results in different responses. This needs more fleshing out, and also reasoning through how processing functions like perSecond(), scaleToSeconds(), etc may affect the decision. -talk more about what happens at each step, how data is manipulated etc +For this reason, this optimization is **experimental** and disabled by default. diff --git a/docs/tools.md b/docs/tools.md index 135dc5a262..a1d2fd4e43 100644 --- a/docs/tools.md +++ b/docs/tools.md @@ -36,6 +36,20 @@ Explains the execution plan for a given query / set of targets Usage: mt-explain + -from string + get data from (inclusive) (default "-24h") + -mdp int + max data points to return (default 800) + -mdp-optimization + enable MaxDataPoints optimization (experimental) + -pre-normalization + enable pre-normalization optimization (default true) + -stable + whether to use only functionality marked as stable (default true) + -time-zone string + time-zone to use for interpreting from/to when needed. (check your config) (default "local") + -to string + get data until (exclusive) (default "now") Example: diff --git a/expr/data_test.go b/expr/data_test.go index b2171b9073..769c751829 100644 --- a/expr/data_test.go +++ b/expr/data_test.go @@ -3,6 +3,7 @@ package expr import ( "math" + "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/schema" ) @@ -247,3 +248,30 @@ func getCopy(in []schema.Point) []schema.Point { copy(out, in) return out } + +func getQuerySeries(patt string, data []schema.Point) models.Series { + return models.Series{ + QueryPatt: patt, + Datapoints: getCopy(data), + Interval: 10, + } +} + +func getTargetSeries(target string, data []schema.Point) models.Series { + return models.Series{ + Target: target, + Datapoints: getCopy(data), + Interval: 10, + } +} + +func getModel(name string, data []schema.Point) models.Series { + series := models.Series{ + Target: name, + QueryPatt: name, + Datapoints: getCopy(data), + Interval: 10, + } + series.SetTags() + return series +} diff --git a/expr/func_aggregate.go b/expr/func_aggregate.go index 91f11d802f..9b9567d83e 100644 --- a/expr/func_aggregate.go +++ b/expr/func_aggregate.go @@ -2,6 +2,7 @@ package expr import ( "strings" + "unsafe" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/schema" @@ -26,6 +27,7 @@ func (s *FuncAggregate) Signature() ([]Arg, []Arg) { } func (s *FuncAggregate) Context(context Context) Context { + context.PNGroup = models.PNGroup(uintptr(unsafe.Pointer(s))) return context } @@ -46,6 +48,7 @@ func (s *FuncAggregate) Exec(cache map[Req][]models.Series) ([]models.Series, er return series, nil } out := pointSlicePool.Get().([]schema.Point) + series = normalize(cache, series) s.agg.function(series, &out) // The tags for the aggregated series is only the tags that are diff --git a/expr/func_aggregate_test.go b/expr/func_aggregate_test.go index 810b2ba802..ce58649b82 100644 --- a/expr/func_aggregate_test.go +++ b/expr/func_aggregate_test.go @@ -16,16 +16,10 @@ func TestAggregateIdentity(t *testing.T) { "average", [][]models.Series{ { - { - QueryPatt: "single", - Datapoints: getCopy(a), - }, + getQuerySeries("single", a), }, }, - models.Series{ - Target: "averageSeries(single)", - Datapoints: getCopy(a), - }, + getTargetSeries("averageSeries(single)", a), t, ) testAggregate( @@ -33,16 +27,10 @@ func TestAggregateIdentity(t *testing.T) { "sum", [][]models.Series{ { - { - QueryPatt: "single", - Datapoints: getCopy(a), - }, + getQuerySeries("single", a), }, }, - models.Series{ - Target: "sumSeries(single)", - Datapoints: getCopy(a), - }, + getTargetSeries("sumSeries(single)", a), t, ) } @@ -52,16 +40,10 @@ func TestAggregateQueryToSingle(t *testing.T) { "average", [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, + getQuerySeries("foo.*", a), }, }, - models.Series{ - Target: "averageSeries(foo.*)", - Datapoints: getCopy(a), - }, + getTargetSeries("averageSeries(foo.*)", a), t, ) } @@ -71,20 +53,11 @@ func TestAggregateMultiple(t *testing.T) { "average", [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, }, - models.Series{ - Target: "averageSeries(foo.*)", - Datapoints: getCopy(avgab), - }, + getTargetSeries("averageSeries(foo.*)", avgab), t, ) testAggregate( @@ -92,20 +65,11 @@ func TestAggregateMultiple(t *testing.T) { "sum", [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, }, - models.Series{ - Target: "sumSeries(foo.*)", - Datapoints: getCopy(sumab), - }, + getTargetSeries("sumSeries(foo.*)", sumab), t, ) testAggregate( @@ -113,40 +77,22 @@ func TestAggregateMultiple(t *testing.T) { "max", [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, }, - models.Series{ - Target: "maxSeries(foo.*)", - Datapoints: getCopy(maxab), - }, + getTargetSeries("maxSeries(foo.*)", maxab), t, ) } func TestAggregateMultipleDiffQuery(t *testing.T) { input := [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, { - { - QueryPatt: "movingAverage(bar, '1min')", - Datapoints: getCopy(c), - }, + getQuerySeries("movingAverage(bar, '1min')", c), }, } @@ -154,30 +100,21 @@ func TestAggregateMultipleDiffQuery(t *testing.T) { "avg-multiple-serieslists", "average", input, - models.Series{ - Target: "averageSeries(foo.*,movingAverage(bar, '1min'))", - Datapoints: getCopy(avgabc), - }, + getTargetSeries("averageSeries(foo.*,movingAverage(bar, '1min'))", avgabc), t, ) testAggregate( "sum-multiple-serieslists", "sum", input, - models.Series{ - Target: "sumSeries(foo.*,movingAverage(bar, '1min'))", - Datapoints: getCopy(sumabc), - }, + getTargetSeries("sumSeries(foo.*,movingAverage(bar, '1min'))", sumabc), t, ) testAggregate( "max-multiple-serieslists", "max", input, - models.Series{ - Target: "maxSeries(foo.*,movingAverage(bar, '1min'))", - Datapoints: getCopy(maxabc), - }, + getTargetSeries("maxSeries(foo.*,movingAverage(bar, '1min'))", maxabc), t, ) } @@ -186,56 +123,32 @@ func TestAggregateMultipleDiffQuery(t *testing.T) { func TestAggregateMultipleTimesSameInput(t *testing.T) { input := [][]models.Series{ { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, { - { - QueryPatt: "foo.*", - Datapoints: getCopy(a), - }, - { - QueryPatt: "foo.*", - Datapoints: getCopy(b), - }, + getQuerySeries("foo.*", a), + getQuerySeries("foo.*", b), }, { - { - QueryPatt: "a", - Datapoints: getCopy(a), - }, + getQuerySeries("a", a), }, { - { - QueryPatt: "a", - Datapoints: getCopy(a), - }, + getQuerySeries("a", a), }, } testAggregate( "avg-multiple-times-same-input", "average", input, - models.Series{ - Target: "averageSeries(foo.*,foo.*,a,a)", - Datapoints: getCopy(avg4a2b), - }, + getTargetSeries("averageSeries(foo.*,foo.*,a,a)", avg4a2b), t, ) testAggregate( "sum-multiple-times-same-input", "sum", input, - models.Series{ - Target: "sumSeries(foo.*,foo.*,a,a)", - Datapoints: getCopy(sum4a2b), - }, + getTargetSeries("sumSeries(foo.*,foo.*,a,a)", sum4a2b), t, ) } diff --git a/expr/func_aspercent.go b/expr/func_aspercent.go index fb5780b70e..96bbde64cd 100644 --- a/expr/func_aspercent.go +++ b/expr/func_aspercent.go @@ -5,6 +5,7 @@ import ( "math" "sort" "strings" + "unsafe" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/errors" @@ -38,6 +39,26 @@ func (s *FuncAsPercent) Signature() ([]Arg, []Arg) { } func (s *FuncAsPercent) Context(context Context) Context { + // when is passing through a PNGroup (or setting one) the right thing? when all series need to be aligned to the same interval + // if we only combine some series with some other series, we don't want coarse series to needlessly coarsen higher resolution data + + // 1) nodes-nil, total single-series -> align all to same interval + // 2) nodes-nil, total multi-series -> match up in pairs (can't be aligned up front) + // 3) nodes-nil, total nil (and not a float) -> align all together + // 4) nodes-nil, total float -> no alignment needed. but pre-existing alignment can remain. + // 5) nodes-non-nil, total nil -> divides groupwise + // 6) nodes non-nil, total serieslist -> divides groupwise + + if s.totalSeries == nil && s.nodes == nil { + // the only scenario where we should introduce a PNGroup is case 3 + if math.IsNaN(s.totalFloat) { + context.PNGroup = models.PNGroup(uintptr(unsafe.Pointer(s))) + } + // in case 4, we can keep a pre-existing PNGroup + } else { + // note: we can't tell the difference between case 1 and 2 up front, so we play it safe and don't align up front + context.PNGroup = 0 + } return context } @@ -135,8 +156,8 @@ func (s *FuncAsPercent) execWithNodes(in, totals []models.Series, cache map[Req] outSeries = append(outSeries, nonesSerie) } else { // key found in both inByKey and totalSerieByKey + serie1, serie2 := normalizeTwo(cache, serie1, totalSerieByKey[key]) serie1 = serie1.Copy(pointSlicePool.Get().([]schema.Point)) - serie2 := totalSerieByKey[key] serie1.QueryPatt = fmt.Sprintf("asPercent(%s,%s)", serie1.QueryPatt, serie2.QueryPatt) serie1.Target = fmt.Sprintf("asPercent(%s,%s)", serie1.Target, serie2.Target) serie1.Tags = map[string]string{"name": serie1.Target} @@ -162,7 +183,7 @@ func (s *FuncAsPercent) execWithoutNodes(in, totals []models.Series, cache map[R var outSeries []models.Series var totalsSerie models.Series if math.IsNaN(s.totalFloat) && totals == nil { - totalsSerie = sumSeries(in, cache) + totalsSerie = sumSeries(normalize(cache, in), cache) if len(in) == 1 { totalsSerie.Target = fmt.Sprintf("sumSeries(%s)", totalsSerie.QueryPatt) totalsSerie.QueryPatt = fmt.Sprintf("sumSeries(%s)", totalsSerie.QueryPatt) @@ -190,19 +211,21 @@ func (s *FuncAsPercent) execWithoutNodes(in, totals []models.Series, cache map[R if len(totals) == len(in) { totalsSerie = totals[i] } - serie = serie.Copy(pointSlicePool.Get().([]schema.Point)) + if len(totalsSerie.Datapoints) > 0 { + serie, totalsSerie = normalizeTwo(cache, serie, totalsSerie) + serie = serie.Copy(pointSlicePool.Get().([]schema.Point)) + for i := range serie.Datapoints { + serie.Datapoints[i].Val = computeAsPercent(serie.Datapoints[i].Val, totalsSerie.Datapoints[i].Val) + } + } else { + serie = serie.Copy(pointSlicePool.Get().([]schema.Point)) + for i := range serie.Datapoints { + serie.Datapoints[i].Val = computeAsPercent(serie.Datapoints[i].Val, s.totalFloat) + } + } serie.QueryPatt = fmt.Sprintf("asPercent(%s,%s)", serie.QueryPatt, totalsSerie.QueryPatt) serie.Target = fmt.Sprintf("asPercent(%s,%s)", serie.Target, totalsSerie.Target) serie.Tags = map[string]string{"name": serie.Target} - for i := range serie.Datapoints { - var totalVal float64 - if len(totalsSerie.Datapoints) > 0 { - totalVal = totalsSerie.Datapoints[i].Val - } else { - totalVal = s.totalFloat - } - serie.Datapoints[i].Val = computeAsPercent(serie.Datapoints[i].Val, totalVal) - } serie.Meta = serie.Meta.Merge(totalsSerie.Meta) outSeries = append(outSeries, serie) cache[Req{}] = append(cache[Req{}], serie) @@ -245,7 +268,7 @@ func getTotalSeries(totalSeriesByKey, inByKey map[string][]models.Series, cache totalSerieByKey := make(map[string]models.Series, len(totalSeriesByKey)) for key := range totalSeriesByKey { if _, ok := inByKey[key]; ok { - totalSerieByKey[key] = sumSeries(totalSeriesByKey[key], cache) + totalSerieByKey[key] = sumSeries(normalize(cache, totalSeriesByKey[key]), cache) } else { totalSerieByKey[key] = totalSeriesByKey[key][0] } @@ -285,6 +308,8 @@ Loop: QueryCons: queryCons, QueryFrom: in[0].QueryFrom, QueryTo: in[0].QueryTo, + QueryMDP: in[0].QueryMDP, + QueryPNGroup: in[0].QueryPNGroup, Tags: map[string]string{"name": name}, Meta: meta, } diff --git a/expr/func_countseries.go b/expr/func_countseries.go index 5dc8d752c7..758bc79186 100644 --- a/expr/func_countseries.go +++ b/expr/func_countseries.go @@ -40,6 +40,8 @@ func (s *FuncCountSeries) Exec(cache map[Req][]models.Series) ([]models.Series, name := fmt.Sprintf("countSeries(%s)", strings.Join(queryPatts, ",")) out := pointSlicePool.Get().([]schema.Point) + // note: if series have different intervals, we could try to be clever and pick the one with highest resolution + // as it's more likely to be useful when combined with other functions, but that's too much hassle l := float64(len(series)) for _, p := range series[0].Datapoints { p.Val = l diff --git a/expr/func_divideseries.go b/expr/func_divideseries.go index efbae2ccb4..e8eb2a7aeb 100644 --- a/expr/func_divideseries.go +++ b/expr/func_divideseries.go @@ -7,8 +7,10 @@ import ( "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/errors" "github.com/grafana/metrictank/schema" + "github.com/grafana/metrictank/util" ) +// FuncDivideSeries divides 1-N dividend series by 1 dividend series type FuncDivideSeries struct { dividend GraphiteFunc divisor GraphiteFunc @@ -26,6 +28,18 @@ func (s *FuncDivideSeries) Signature() ([]Arg, []Arg) { } func (s *FuncDivideSeries) Context(context Context) Context { + // note: technically divideSeries() is a sort of aggregation function + // that "aggregates" each dividend series together with the divisor. + // thus, it's theoretically possible to apply pre-normalization + // but, if it receives dividend series that may differ in their interval, then + // this would be really hard to juggle because the divisor would get used + // multiple times with different intervals. + // so to be safe, let's just treat divideSeries like an opaque aggregation and + // cancel out ongoing pre-normalization. + // we wouldn't want to pre-normalize all dividends and the divisor to 1 common + // interval because that could cause coarse dividends to affect (coarsen) + // their fellow dividends + context.PNGroup = 0 return context } @@ -41,11 +55,30 @@ func (s *FuncDivideSeries) Exec(cache map[Req][]models.Series) ([]models.Series, if len(divisors) != 1 { return nil, errors.NewBadRequestf("need 1 divisor series, not %d", len(divisors)) } - divisor := divisors[0] var series []models.Series + // if len(dividends) > 1 the same divisor series will be used multiple times, + // and we'll possibly need to normalize it to different intervals (if the dividends have differing intervals) + // (we also need to normalize if there's only 1 dividend but it has a different interval than the divisor) + // so let's keep track of the different "versions" of the divisor that we have available. + // (the dividend(s) may also need to be normalized but we only use them once so they don't require special attention) + divisorsByRes := make(map[uint32]models.Series) + divisorsByRes[divisors[0].Interval] = divisors[0] for _, dividend := range dividends { out := pointSlicePool.Get().([]schema.Point) + divisor := divisors[0] + if dividend.Interval != divisors[0].Interval { + lcm := util.Lcm([]uint32{dividend.Interval, divisor.Interval}) + newDiv, ok := divisorsByRes[lcm] + if ok { + divisor = newDiv + // we now have the right divisor but may still need to normalize the dividend + dividend, divisor = normalizeTwo(cache, dividend, divisor) + } else { + dividend, divisor = normalizeTwo(cache, dividend, divisor) + divisorsByRes[lcm] = divisor + } + } for i := 0; i < len(dividend.Datapoints); i++ { p := schema.Point{ Ts: dividend.Datapoints[i].Ts, @@ -69,6 +102,8 @@ func (s *FuncDivideSeries) Exec(cache map[Req][]models.Series) ([]models.Series, QueryCons: dividend.QueryCons, QueryFrom: dividend.QueryFrom, QueryTo: dividend.QueryTo, + QueryPNGroup: dividend.QueryPNGroup, + QueryMDP: dividend.QueryMDP, Meta: dividend.Meta.Copy().Merge(divisor.Meta), } cache[Req{}] = append(cache[Req{}], output) diff --git a/expr/func_divideserieslists.go b/expr/func_divideserieslists.go index eb39fecd1e..398145f3aa 100644 --- a/expr/func_divideserieslists.go +++ b/expr/func_divideserieslists.go @@ -9,6 +9,7 @@ import ( "github.com/grafana/metrictank/schema" ) +// FuncDivideSeriesLists divides dividends by divisors, pairwise type FuncDivideSeriesLists struct { dividends GraphiteFunc divisors GraphiteFunc @@ -26,6 +27,13 @@ func (s *FuncDivideSeriesLists) Signature() ([]Arg, []Arg) { } func (s *FuncDivideSeriesLists) Context(context Context) Context { + // note: like FuncDivideSeries, this is an aggregation function (turning pairs of series into one) + // unlike FuncDivideSeries, we don't use any input series more than once, + // thus any already proposed pre-normalization can proceed as planned + // and hence do not have to reset PNGroup. + // if anything, in some exotic cases divisors (and dividends) may have different intervals amongst themselves + // but matching intervals when we pair up a divisor with a dividend, in which case we could technically introduce pre-normalization + // but we can't really predict that here, so let's not worry about that. return context } @@ -43,8 +51,9 @@ func (s *FuncDivideSeriesLists) Exec(cache map[Req][]models.Series) ([]models.Se } var series []models.Series - for i, dividend := range dividends { - divisor := divisors[i] + for i := range dividends { + dividend, divisor := normalizeTwo(cache, dividends[i], divisors[i]) + out := pointSlicePool.Get().([]schema.Point) for i := 0; i < len(dividend.Datapoints); i++ { p := schema.Point{ @@ -69,6 +78,8 @@ func (s *FuncDivideSeriesLists) Exec(cache map[Req][]models.Series) ([]models.Se QueryCons: dividend.QueryCons, QueryFrom: dividend.QueryFrom, QueryTo: dividend.QueryTo, + QueryMDP: dividend.QueryMDP, + QueryPNGroup: dividend.QueryPNGroup, Meta: dividend.Meta.Copy().Merge(divisor.Meta), } cache[Req{}] = append(cache[Req{}], output) diff --git a/expr/func_groupbytags.go b/expr/func_groupbytags.go index 7a5f20c1cc..b5a4427cc6 100644 --- a/expr/func_groupbytags.go +++ b/expr/func_groupbytags.go @@ -28,6 +28,7 @@ func (s *FuncGroupByTags) Signature() ([]Arg, []Arg) { } func (s *FuncGroupByTags) Context(context Context) Context { + context.PNGroup = 0 return context } @@ -124,12 +125,14 @@ func (s *FuncGroupByTags) Exec(cache map[Req][]models.Series) ([]models.Series, QueryCons: queryCons, QueryFrom: group.s[0].QueryFrom, QueryTo: group.s[0].QueryTo, + QueryMDP: group.s[0].QueryMDP, + QueryPNGroup: group.s[0].QueryPNGroup, Meta: group.m, } newSeries.SetTags() newSeries.Datapoints = pointSlicePool.Get().([]schema.Point) - + group.s = normalize(cache, group.s) aggFunc(group.s, &newSeries.Datapoints) cache[Req{}] = append(cache[Req{}], newSeries) diff --git a/expr/func_groupbytags_test.go b/expr/func_groupbytags_test.go index 8e552332f7..c4961506d0 100644 --- a/expr/func_groupbytags_test.go +++ b/expr/func_groupbytags_test.go @@ -12,16 +12,6 @@ import ( "github.com/grafana/metrictank/test" ) -func getModel(name string, data []schema.Point) models.Series { - series := models.Series{ - Target: name, - QueryPatt: name, - Datapoints: getCopy(data), - } - series.SetTags() - return series -} - // Test error cases func TestNoTags(t *testing.T) { in := []models.Series{ diff --git a/expr/func_persecond.go b/expr/func_persecond.go index 60a583ffa2..f6397c217c 100644 --- a/expr/func_persecond.go +++ b/expr/func_persecond.go @@ -59,12 +59,14 @@ func (s *FuncPerSecond) Exec(cache map[Req][]models.Series) ([]models.Series, er } } s := models.Series{ - Target: fmt.Sprintf("perSecond(%s)", serie.Target), - QueryPatt: fmt.Sprintf("perSecond(%s)", serie.QueryPatt), - Tags: serie.Tags, - Datapoints: out, - Interval: serie.Interval, - Meta: serie.Meta, + Target: fmt.Sprintf("perSecond(%s)", serie.Target), + QueryPatt: fmt.Sprintf("perSecond(%s)", serie.QueryPatt), + Tags: serie.Tags, + Datapoints: out, + Interval: serie.Interval, + Meta: serie.Meta, + QueryMDP: serie.QueryMDP, + QueryPNGroup: serie.QueryPNGroup, } outputs = append(outputs, s) cache[Req{}] = append(cache[Req{}], s) diff --git a/expr/func_smartsummarize.go b/expr/func_smartsummarize.go index 841d2fd150..6b62b8823f 100644 --- a/expr/func_smartsummarize.go +++ b/expr/func_smartsummarize.go @@ -23,6 +23,8 @@ func (s *FuncSmartSummarize) Signature() ([]Arg, []Arg) { } func (s *FuncSmartSummarize) Context(context Context) Context { + context.MDP = 0 + context.PNGroup = 0 context.consol = 0 return context } diff --git a/expr/func_summarize.go b/expr/func_summarize.go index 30aea8ab1c..54f23c9081 100644 --- a/expr/func_summarize.go +++ b/expr/func_summarize.go @@ -32,6 +32,8 @@ func (s *FuncSummarize) Signature() ([]Arg, []Arg) { } func (s *FuncSummarize) Context(context Context) Context { + context.MDP = 0 + context.PNGroup = 0 context.consol = 0 return context } @@ -68,14 +70,16 @@ func (s *FuncSummarize) Exec(cache map[Req][]models.Series) ([]models.Series, er out := summarizeValues(serie, aggFunc, interval, newStart, newEnd) output := models.Series{ - Target: newName(serie.Target), - QueryPatt: newName(serie.QueryPatt), - QueryFrom: serie.QueryFrom, - QueryTo: serie.QueryTo, - Tags: serie.CopyTagsWith("summarize", s.intervalString), - Datapoints: out, - Interval: interval, - Meta: serie.Meta, + Target: newName(serie.Target), + QueryPatt: newName(serie.QueryPatt), + QueryFrom: serie.QueryFrom, + QueryTo: serie.QueryTo, + QueryMDP: serie.QueryMDP, + QueryPNGroup: serie.QueryPNGroup, + Tags: serie.CopyTagsWith("summarize", s.intervalString), + Datapoints: out, + Interval: interval, + Meta: serie.Meta, } output.Tags["summarizeFunction"] = s.fn diff --git a/expr/funcs.go b/expr/funcs.go index 0e986c75d8..ff02d96e61 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -7,9 +7,12 @@ import ( // Context describes a series timeframe and consolidator type Context struct { - from uint32 - to uint32 - consol consolidation.Consolidator // can be 0 to mean undefined + from uint32 + to uint32 + consol consolidation.Consolidator // can be 0 to mean undefined + PNGroup models.PNGroup // pre-normalization group. if the data can be safely pre-normalized + MDP uint32 // if we can MDP-optimize, reflects runtime consolidation MaxDataPoints. 0 otherwise + optimizations Optimizations } // GraphiteFunc defines a graphite processing function diff --git a/expr/normalize.go b/expr/normalize.go new file mode 100644 index 0000000000..0ff34db93c --- /dev/null +++ b/expr/normalize.go @@ -0,0 +1,90 @@ +package expr + +import ( + "fmt" + "math" + + "github.com/grafana/metrictank/api/models" + "github.com/grafana/metrictank/consolidation" + "github.com/grafana/metrictank/schema" + "github.com/grafana/metrictank/util" +) + +// normalize normalizes series to the same common LCM interval - if they don't already have the same interval +// any adjusted series gets created in a series drawn out of the pool and is added to the cache so it can be reclaimed +func normalize(cache map[Req][]models.Series, in []models.Series) []models.Series { + var intervals []uint32 + for _, s := range in { + if s.Interval == 0 { + panic("illegal interval 0") + } + intervals = append(intervals, s.Interval) + } + lcm := util.Lcm(intervals) + for i, s := range in { + if s.Interval != lcm { + in[i] = normalizeTo(cache, s, lcm) + } + } + return in +} + +func normalizeTwo(cache map[Req][]models.Series, a, b models.Series) (models.Series, models.Series) { + if a.Interval == b.Interval { + return a, b + } + intervals := []uint32{a.Interval, b.Interval} + lcm := util.Lcm(intervals) + + if a.Interval != lcm { + a = normalizeTo(cache, a, lcm) + } + if b.Interval != lcm { + b = normalizeTo(cache, b, lcm) + } + return a, b +} + +// normalizeTo normalizes the given series to the desired interval +// the following MUST be true when calling this: +// * interval > in.Interval +// * interval % in.Interval == 0 +func normalizeTo(cache map[Req][]models.Series, in models.Series, interval uint32) models.Series { + // we need to copy the datapoints first because the consolidater will reuse the input slice + // also, the input may not be pre-canonical. so add nulls in front and at the back to make it pre-canonical. + // this may make points in front and at the back less accurate when consolidated (e.g. summing when some of the points are null results in a lower value) + // but this is what graphite does.... + datapoints := pointSlicePool.Get().([]schema.Point) + + if len(in.Datapoints) == 0 { + panic(fmt.Sprintf("series %q cannot be normalized from interval %d to %d because it is empty", in.Target, in.Interval, interval)) + } + + // example of how this works: + // if in.Interval is 5, and interval is 15, then for example, to generate point 15, you want inputs 5, 10 and 15. + // or more generally (you can follow any example vertically): + // 5 10 15 20 25 30 35 40 45 50 <-- if any of these timestamps are your first point in `in` + // 5 5 5 20 20 20 35 35 35 50 <-- then these are the corresponding timestamps of the first values we want as input for the consolidator + // 15 15 15 30 30 30 45 45 45 60 <-- which, when fed through alignForward(), result in these numbers + // 5 5 5 20 20 20 35 35 35 50 <-- subtract (aggnum-1)* in.interval or equivalent -interval + in.Interval = -15 + 5 = -10. these are our desired numbers! + + for ts := alignForward(in.Datapoints[0].Ts, interval) - interval + in.Interval; ts < in.Datapoints[0].Ts; ts += interval { + datapoints = append(datapoints, schema.Point{Val: math.NaN(), Ts: ts}) + } + + datapoints = append(datapoints, in.Datapoints...) + + in.Datapoints = consolidation.Consolidate(datapoints, interval/in.Interval, in.Consolidator) + in.Interval = interval + cache[Req{}] = append(cache[Req{}], in) + return in +} + +// alignForward aligns ts to the next timestamp that divides by the interval, except if it is already aligned +func alignForward(ts, interval uint32) uint32 { + remain := ts % interval + if remain == 0 { + return ts + } + return ts + interval - remain +} diff --git a/expr/normalize_test.go b/expr/normalize_test.go new file mode 100644 index 0000000000..201e6ec57f --- /dev/null +++ b/expr/normalize_test.go @@ -0,0 +1,135 @@ +package expr + +import ( + "fmt" + "testing" + + "github.com/google/go-cmp/cmp" + "github.com/grafana/metrictank/api/models" + "github.com/grafana/metrictank/consolidation" + "github.com/grafana/metrictank/schema" +) + +func TestNormalizeOneSeriesAdjustWithPreCanonicalize(t *testing.T) { + in := []models.Series{ + { + Interval: 5, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + // ts=5 null will be added to make pre-canonical wrt to 10 + {Ts: 10, Val: 10}, + {Ts: 15, Val: 15}, + {Ts: 20, Val: 20}, + }, + }, + { + Interval: 10, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 10, Val: 10}, + {Ts: 20, Val: 20}, + }, + }, + } + want := []models.Series{ + { + Interval: 10, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 10, Val: 10}, + {Ts: 20, Val: 35}, + }, + }, + { + Interval: 10, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 10, Val: 10}, + {Ts: 20, Val: 20}, + }, + }, + } + cache := make(map[Req][]models.Series) + got := normalize(cache, in) + fmt.Println("got:") + fmt.Println(got[0].Datapoints) + fmt.Println(got[1].Datapoints) + if diff := cmp.Diff(want, got); diff != "" { + t.Errorf("TestNormalize() mismatch (-want +got):\n%s", diff) + } +} + +// LCM is 60, which means we need to generate points with ts=60 and ts=120 +func TestNormalizeMultiLCMSeriesAdjustWithPreCanonicalize(t *testing.T) { + in := []models.Series{ + { + Interval: 15, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 15, Val: 15}, + {Ts: 30, Val: 30}, + {Ts: 45, Val: 45}, + {Ts: 60, Val: 60}, + {Ts: 75, Val: 75}, + }, + }, + { + Interval: 20, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 20, Val: 20}, + {Ts: 40, Val: 40}, + {Ts: 60, Val: 60}, + {Ts: 80, Val: 80}, + }, + }, + { + Interval: 10, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + // ts 10 will be added to make pre-canonical wrt to 60 + {Ts: 20, Val: 20}, + {Ts: 30, Val: 30}, + {Ts: 40, Val: 40}, + {Ts: 50, Val: 50}, + {Ts: 60, Val: 60}, + {Ts: 70, Val: 70}, + {Ts: 80, Val: 80}, + }, + }, + } + want := []models.Series{ + { + Interval: 60, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 60, Val: 150}, + {Ts: 120, Val: 75}, + }, + }, + { + Interval: 60, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 60, Val: 120}, + {Ts: 120, Val: 80}, + }, + }, + { + Interval: 60, + Consolidator: consolidation.Sum, + Datapoints: []schema.Point{ + {Ts: 60, Val: 200}, + {Ts: 120, Val: 150}, + }, + }, + } + cache := make(map[Req][]models.Series) + got := normalize(cache, in) + fmt.Println("got:") + fmt.Println(got[0].Datapoints) + fmt.Println(got[1].Datapoints) + if diff := cmp.Diff(want, got); diff != "" { + t.Errorf("TestNormalize() mismatch (-want +got):\n%s", diff) + } +} diff --git a/expr/parse.go b/expr/parse.go index c9f3f4e71e..47a2682da9 100644 --- a/expr/parse.go +++ b/expr/parse.go @@ -441,9 +441,8 @@ func extractMetric(m string) string { return m[start:end] } -// aggKey creates a key for a serie based on its target metric as well -// as a list of nodes which need to be extracted -// returns a single string +// aggKey returns a string key by applying the selectors +// (integers for node positions or strings for tag names) to the given serie func aggKey(serie models.Series, nodes []expr) string { metric := extractMetric(serie.Target) if len(metric) == 0 { diff --git a/expr/plan.go b/expr/plan.go index dd54727419..5e24be7369 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -3,28 +3,105 @@ package expr import ( "fmt" "io" + "strings" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/consolidation" "github.com/grafana/metrictank/errors" ) +type Optimizations struct { + PreNormalization bool + MDP bool +} + +func (o Optimizations) ApplyUserPrefs(s string) (Optimizations, error) { + // no user override. stick to what we have + if s == "" { + return o, nil + } + // user passed an override. it's either 'none' (no optimizations) or a list of the ones that should be enabled + o.PreNormalization = false + o.MDP = false + if s == "none" { + return o, nil + } + prefs := strings.Split(s, ",") + for _, pref := range prefs { + switch pref { + case "pn": + o.PreNormalization = true + case "mdp": + o.MDP = true + default: + return o, fmt.Errorf("unrecognized optimization %q", pref) + } + } + return o, nil +} + // Req represents a request for one/more series type Req struct { - Query string // whatever was parsed as the query out of a graphite target. e.g. target=sum(foo.{b,a}r.*) -> foo.{b,a}r.* -> this will go straight to index lookup - From uint32 - To uint32 - Cons consolidation.Consolidator // can be 0 to mean undefined + Query string // whatever was parsed as the query out of a graphite target. e.g. target=sum(foo.{b,a}r.*) -> foo.{b,a}r.* -> this will go straight to index lookup + From uint32 + To uint32 + Cons consolidation.Consolidator // can be 0 to mean undefined + PNGroup models.PNGroup + MDP uint32 // if we can MDP-optimize, reflects runtime consolidation MaxDataPoints. 0 otherwise. } // NewReq creates a new Req. pass cons=0 to leave consolidator undefined, // leaving up to the caller (in graphite's case, it would cause a lookup into storage-aggregation.conf) -func NewReq(query string, from, to uint32, cons consolidation.Consolidator) Req { +func NewReq(query string, from, to uint32, cons consolidation.Consolidator, PNGroup models.PNGroup, MDP uint32) Req { return Req{ + Query: query, + From: from, + To: to, + Cons: cons, + PNGroup: PNGroup, + MDP: MDP, + } +} + +func NewReqFromContext(query string, c Context) Req { + r := Req{ Query: query, - From: from, - To: to, - Cons: cons, + From: c.from, + To: c.to, + Cons: c.consol, + } + if c.optimizations.PreNormalization { + r.PNGroup = c.PNGroup + } + if c.optimizations.MDP { + r.MDP = c.MDP + } + return r +} + +// NewReqFromSeries generates a Req back from a series +// a models.Series has all the properties attached to it +// to find out which Req it came from +func NewReqFromSerie(serie models.Series) Req { + return Req{ + Query: serie.QueryPatt, + From: serie.QueryFrom, + To: serie.QueryTo, + Cons: serie.QueryCons, + PNGroup: serie.QueryPNGroup, + MDP: serie.QueryMDP, + } + +} + +func (r Req) ToModel() models.Req { + return models.Req{ + Pattern: r.Query, + From: r.From, + To: r.To, + MaxPoints: r.MDP, + PNGroup: r.PNGroup, + ConsReq: r.Cons, } } @@ -48,8 +125,20 @@ func (p Plan) Dump(w io.Writer) { fmt.Fprintln(w, e.Print(2)) } fmt.Fprintf(w, "* Reqs:\n") + + maxQueryLen := 5 for _, r := range p.Reqs { - fmt.Fprintln(w, " ", r) + if len(r.Query) > maxQueryLen { + maxQueryLen = len(r.Query) + } + } + // ! PNGroups are pointers which can be upto 21 characters long on 64bit + headPatt := fmt.Sprintf("%%%ds %%12s %%12s %%25s %%21s %%6s\n", maxQueryLen) + linePatt := fmt.Sprintf("%%%ds %%12d %%12d %%25s %%21d %%6d\n", maxQueryLen) + fmt.Fprintf(w, headPatt, "query", "from", "to", "consolidator", "PNGroup", "MDP") + + for _, r := range p.Reqs { + fmt.Fprintf(w, linePatt, r.Query, r.From, r.To, r.Cons, r.PNGroup, r.MDP) } fmt.Fprintf(w, "MaxDataPoints: %d\n", p.MaxDataPoints) fmt.Fprintf(w, "From: %d\n", p.From) @@ -63,29 +152,29 @@ func (p Plan) Dump(w io.Writer) { // * validation of arguments // * allow functions to modify the Context (change data range or consolidation) // * future version: allow functions to mark safe to pre-aggregate using consolidateBy or not -func NewPlan(exprs []*expr, from, to, mdp uint32, stable bool, reqs []Req) (Plan, error) { - var err error - var funcs []GraphiteFunc +func NewPlan(exprs []*expr, from, to, mdp uint32, stable bool, optimizations Optimizations) (Plan, error) { + plan := Plan{ + exprs: exprs, + MaxDataPoints: mdp, + From: from, + To: to, + } for _, e := range exprs { - var fn GraphiteFunc context := Context{ - from: from, - to: to, + from: from, + to: to, + MDP: mdp, + PNGroup: 0, // making this explicit here for easy code grepping + optimizations: optimizations, } - fn, reqs, err = newplan(e, context, stable, reqs) + fn, reqs, err := newplan(e, context, stable, plan.Reqs) if err != nil { return Plan{}, err } - funcs = append(funcs, fn) + plan.Reqs = reqs + plan.funcs = append(plan.funcs, fn) } - return Plan{ - Reqs: reqs, - exprs: exprs, - funcs: funcs, - MaxDataPoints: mdp, - From: from, - To: to, - }, nil + return plan, nil } // newplan adds requests as needed for the given expr, resolving function calls as needed @@ -94,7 +183,7 @@ func newplan(e *expr, context Context, stable bool, reqs []Req) (GraphiteFunc, [ return nil, nil, errors.NewBadRequest("request must be a function call or metric pattern") } if e.etype == etName { - req := NewReq(e.str, context.from, context.to, context.consol) + req := NewReqFromContext(e.str, context) reqs = append(reqs, req) return NewGet(req), reqs, nil } else if e.etype == etFunc && e.str == "seriesByTag" { @@ -104,7 +193,7 @@ func newplan(e *expr, context Context, stable bool, reqs []Req) (GraphiteFunc, [ // string back into the Query member of a new request to be parsed later. // TODO - find a way to prevent this parse/encode/parse/encode loop expressionStr := "seriesByTag(" + e.argsStr + ")" - req := NewReq(expressionStr, context.from, context.to, context.consol) + req := NewReqFromContext(expressionStr, context) reqs = append(reqs, req) return NewGet(req), reqs, nil } diff --git a/expr/plan_test.go b/expr/plan_test.go index cbe5119347..f9d85dcf25 100644 --- a/expr/plan_test.go +++ b/expr/plan_test.go @@ -5,6 +5,7 @@ import ( "reflect" "testing" + "github.com/google/go-cmp/cmp" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/consolidation" ) @@ -32,7 +33,7 @@ func TestArgs(t *testing.T) { }, nil, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -46,7 +47,7 @@ func TestArgs(t *testing.T) { }, nil, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -60,7 +61,7 @@ func TestArgs(t *testing.T) { }, nil, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -75,7 +76,7 @@ func TestArgs(t *testing.T) { "alignToFrom": {etype: etBool, bool: true}, }, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -90,7 +91,7 @@ func TestArgs(t *testing.T) { "alignToFrom": {etype: etString, str: "true"}, }, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -105,7 +106,7 @@ func TestArgs(t *testing.T) { "alignToFrom": {etype: etBool, bool: true}, }, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -147,7 +148,7 @@ func TestArgs(t *testing.T) { "func": {etype: etString, str: "sum"}, }, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -161,7 +162,7 @@ func TestArgs(t *testing.T) { "alignToFrom": {etype: etBool, bool: true}, }, []Req{ - NewReq("foo.bar.*", from, to, 0), + NewReq("foo.bar.*", from, to, 0, 0, 0), }, nil, }, @@ -438,6 +439,151 @@ func TestArgInIntKeyword(t *testing.T) { } } +// TestOptimizationFlags tests that the optimization (PNGroups and MDP for MDP-optimization) flags are +// set in line with the optimization settings passed to the planner. +func TestOptimizationFlags(t *testing.T) { + from := uint32(1000) + to := uint32(2000) + stable := true + + // note, use the number 1 to mean "a PNGroup". these tests currently don't support multiple PNGroups. + type testCase struct { + in string + wantReq []Req + } + + compare := func(i int, opts Optimizations, c testCase, plan Plan, err error) { + // for simplicity, test cases declare 1 to mean "a PNGroup" + for i, req := range plan.Reqs { + if req.PNGroup > 0 { + plan.Reqs[i].PNGroup = 1 + } + } + + if diff := cmp.Diff(c.wantReq, plan.Reqs); diff != "" { + t.Errorf("case %d: %q with opts %+v (-want +got):\n%s", i, c.in, opts, diff) + } + } + + cases := []testCase{ + { + // no transparent aggregation so don't align the data. Though, could be MDP optimized + "a", + []Req{ + NewReq("a", from, to, 0, 0, 800), + }, + }, + { + "summarize(a,'1h')", // greedy resolution function. disables MDP optimizations + []Req{ + NewReq("a", from, to, 0, 0, 0), + }, + }, + { + "sum(a)", // transparent aggregation. enables PN-optimization + []Req{ + NewReq("a", from, to, 0, 1, 800), + }, + }, + { + "summarize(sum(a),'1h')", + []Req{ + NewReq("a", from, to, 0, 1, 0), + }, + }, + { + // a will go through some functions that don't matter, then hits a transparent aggregation + "summarize(sum(perSecond(min(scale(a,1)))),'1h')", + []Req{ + NewReq("a", from, to, 0, 1, 0), + }, + }, + { + // a is not PN-optimizable due to the opaque aggregation, whereas b is thanks to the transparent aggregation, that they hit first. + "sum(group(groupByTags(a,'sum','foo'), avg(b)))", + []Req{ + NewReq("a", from, to, 0, 0, 800), + NewReq("b", from, to, 0, 1, 800), + }, + }, + { + // a is not PN-optimizable because it doesn't go through a transparent aggregation + // b is, because it hits a transparent aggregation before it hits the opaque aggregation + // c is neither PN-optimizable, nor MDP-optimizable, because it hits an interval altering + GR function, before it hits anything else + "groupByTags(group(groupByTags(a,'sum','tag'), avg(b), avg(summarize(c,'1h'))),'sum','tag2')", + []Req{ + NewReq("a", from, to, 0, 0, 800), + NewReq("b", from, to, 0, 1, 800), + NewReq("c", from, to, 0, 0, 0), + }, + }, + } + for i, c := range cases { + // make a pristine copy of the data such that we can tweak it for different scenarios + origWantReqs := make([]Req, len(c.wantReq)) + copy(origWantReqs, c.wantReq) + + // first, try with all optimizations: + opts := Optimizations{ + PreNormalization: true, + MDP: true, + } + exprs, err := ParseMany([]string{c.in}) + if err != nil { + t.Fatal(err) + } + plan, err := NewPlan(exprs, from, to, 800, stable, opts) + if err != nil { + t.Fatal(err) + } + compare(i, opts, c, plan, err) + + // now, disable MDP. This should result simply in disabling all MDP flags on all requests + opts.MDP = false + c.wantReq = make([]Req, len(origWantReqs)) + copy(c.wantReq, origWantReqs) + for j := range c.wantReq { + c.wantReq[j].MDP = 0 + } + + plan, err = NewPlan(exprs, from, to, 800, stable, opts) + if err != nil { + t.Fatal(err) + } + compare(i, opts, c, plan, err) + + // now disable (only) PN-optimizations. This should result simply in turning off all PNGroups + opts.MDP = true + opts.PreNormalization = false + c.wantReq = make([]Req, len(origWantReqs)) + copy(c.wantReq, origWantReqs) + for j := range c.wantReq { + c.wantReq[j].PNGroup = 0 + } + plan, err = NewPlan(exprs, from, to, 800, stable, opts) + if err != nil { + t.Fatal(err) + } + compare(i, opts, c, plan, err) + + // now disable both optimizations at the same time + opts.MDP = false + opts.PreNormalization = false + c.wantReq = make([]Req, len(origWantReqs)) + copy(c.wantReq, origWantReqs) + for j := range c.wantReq { + c.wantReq[j].MDP = 0 + c.wantReq[j].PNGroup = 0 + } + + plan, err = NewPlan(exprs, from, to, 800, stable, opts) + if err != nil { + t.Fatal(err) + } + compare(i, opts, c, plan, err) + } +} + // TestConsolidateBy tests for a variety of input targets, wether consolidateBy settings are correctly // propagated down the tree (to fetch requests) and up the tree (to runtime consolidation of the output) func TestConsolidateBy(t *testing.T) { @@ -453,7 +599,7 @@ func TestConsolidateBy(t *testing.T) { { "a", []Req{ - NewReq("a", from, to, 0), + NewReq("a", from, to, 0, 0, 0), }, nil, []models.Series{ @@ -464,7 +610,7 @@ func TestConsolidateBy(t *testing.T) { // consolidation flows both up and down the tree `consolidateBy(a, "sum")`, []Req{ - NewReq("a", from, to, consolidation.Sum), + NewReq("a", from, to, consolidation.Sum, 0, 0), }, nil, []models.Series{ @@ -475,7 +621,7 @@ func TestConsolidateBy(t *testing.T) { // wrap with regular function -> consolidation goes both up and down `scale(consolidateBy(a, "sum"),1)`, []Req{ - NewReq("a", from, to, consolidation.Sum), + NewReq("a", from, to, consolidation.Sum, 0, 0), }, nil, []models.Series{ @@ -486,7 +632,7 @@ func TestConsolidateBy(t *testing.T) { // wrapping by a special function does not affect fetch consolidation, but resets output consolidation `perSecond(consolidateBy(a, "sum"))`, []Req{ - NewReq("a", from, to, consolidation.Sum), + NewReq("a", from, to, consolidation.Sum, 0, 0), }, nil, []models.Series{ @@ -497,7 +643,7 @@ func TestConsolidateBy(t *testing.T) { // consolidation setting streams down and up unaffected by scale `consolidateBy(scale(a, 1), "sum")`, []Req{ - NewReq("a", from, to, consolidation.Sum), + NewReq("a", from, to, consolidation.Sum, 0, 0), }, nil, []models.Series{ @@ -508,7 +654,7 @@ func TestConsolidateBy(t *testing.T) { // perSecond changes data semantics, fetch consolidation should be reset to default `consolidateBy(perSecond(a), "sum")`, []Req{ - NewReq("a", from, to, 0), + NewReq("a", from, to, 0, 0, 0), }, nil, []models.Series{ @@ -520,8 +666,8 @@ func TestConsolidateBy(t *testing.T) { // TODO: I think it can be argued that the max here is only intended for the output, not to the inputs `consolidateBy(divideSeries(consolidateBy(a, "min"), b), "max")`, []Req{ - NewReq("a", from, to, consolidation.Min), - NewReq("b", from, to, consolidation.Max), + NewReq("a", from, to, consolidation.Min, 0, 0), + NewReq("b", from, to, consolidation.Max, 0, 0), }, nil, []models.Series{ @@ -532,8 +678,8 @@ func TestConsolidateBy(t *testing.T) { // data should be requested with fetch consolidation min, but runtime consolidation max `consolidateBy(sumSeries(consolidateBy(a, "min"), b), "max")`, []Req{ - NewReq("a", from, to, consolidation.Min), - NewReq("b", from, to, consolidation.Max), + NewReq("a", from, to, consolidation.Min, 0, 0), + NewReq("b", from, to, consolidation.Max, 0, 0), }, nil, []models.Series{ @@ -545,36 +691,40 @@ func TestConsolidateBy(t *testing.T) { for i, c := range cases { // for the purpose of this test, we assume ParseMany works fine. exprs, _ := ParseMany([]string{c.in}) - plan, err := NewPlan(exprs, from, to, 800, stable, nil) + plan, err := NewPlan(exprs, from, to, 800, stable, Optimizations{}) if err != nil { t.Fatal(err) } if !reflect.DeepEqual(err, c.expErr) { t.Errorf("case %d: %q, expected error %v - got %v", i, c.in, c.expErr, err) } - if !reflect.DeepEqual(plan.Reqs, c.expReq) { - t.Errorf("case %d: %q, expected req %v - got %v", i, c.in, c.expReq, plan.Reqs) + if diff := cmp.Diff(c.expReq, plan.Reqs); diff != "" { + t.Errorf("case %d: %q (-want +got):\n%s", i, c.in, diff) } input := map[Req][]models.Series{ - NewReq("a", from, to, 0): {{ + NewReq("a", from, to, 0, 0, 0): {{ QueryPatt: "a", Target: "a", Consolidator: consolidation.Avg, // emulate the fact that a by default will use avg + Interval: 10, }}, - NewReq("a", from, to, consolidation.Min): {{ + NewReq("a", from, to, consolidation.Min, 0, 0): {{ QueryPatt: "a", Target: "a", Consolidator: consolidation.Min, + Interval: 10, }}, - NewReq("a", from, to, consolidation.Sum): {{ + NewReq("a", from, to, consolidation.Sum, 0, 0): {{ QueryPatt: "a", Target: "a", Consolidator: consolidation.Sum, + Interval: 10, }}, - NewReq("b", from, to, consolidation.Max): {{ + NewReq("b", from, to, consolidation.Max, 0, 0): {{ QueryPatt: "b", Target: "b", Consolidator: consolidation.Max, + Interval: 10, }}, } out, err := plan.Run(input) @@ -715,7 +865,7 @@ func TestNamingChains(t *testing.T) { if err != nil { t.Fatal(err) } - plan, err := NewPlan(exprs, from, to, 800, stable, nil) + plan, err := NewPlan(exprs, from, to, 800, stable, Optimizations{}) if err != nil { t.Fatal(err) } @@ -726,6 +876,7 @@ func TestNamingChains(t *testing.T) { series[j] = models.Series{ QueryPatt: plan.Reqs[0].Query, Target: key, + Interval: 10, } } input := map[Req][]models.Series{ @@ -820,7 +971,7 @@ func TestTargetErrors(t *testing.T) { if err != c.expectedParseError { t.Fatalf("case %q: expected parse error %q but got %q", c.testDescription, c.expectedParseError, err) } - _, err = NewPlan(exprs, from, to, 800, stable, nil) + _, err = NewPlan(exprs, from, to, 800, stable, Optimizations{}) if err != c.expectedPlanError { t.Fatalf("case %q: expected plan error %q but got %q", c.testDescription, c.expectedPlanError, err) } diff --git a/metrictank-sample.ini b/metrictank-sample.ini index 00184eacce..fd0cdf65aa 100644 --- a/metrictank-sample.ini +++ b/metrictank-sample.ini @@ -248,6 +248,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/scripts/config/metrictank-docker.ini b/scripts/config/metrictank-docker.ini index 0e383d3403..aee15c6f25 100644 --- a/scripts/config/metrictank-docker.ini +++ b/scripts/config/metrictank-docker.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/scripts/config/metrictank-package.ini b/scripts/config/metrictank-package.ini index f72de06709..baefe0527f 100644 --- a/scripts/config/metrictank-package.ini +++ b/scripts/config/metrictank-package.ini @@ -244,6 +244,10 @@ get-targets-concurrency = 20 tagdb-default-limit = 100 # ratio of peer responses after which speculative querying (aka spec-exec) is used. Set to 1 to disable. speculation-threshold = 1 +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## diff --git a/test/key.go b/test/key.go index 6f7356a936..abf9927cb3 100644 --- a/test/key.go +++ b/test/key.go @@ -20,6 +20,10 @@ func GetMKey(suffix int) schema.MKey { } } +func KeyToInt(key schema.MKey) int { + return int(uint32(key.Key[15]) + uint32(key.Key[14])<<8 + uint32(key.Key[13])<<16 + uint32(key.Key[12])<<24) +} + func MustMKeyFromString(id string) schema.MKey { mkey, err := schema.MKeyFromString(id) if err != nil { diff --git a/util/combinations.go b/util/combinations.go new file mode 100644 index 0000000000..d5125ae689 --- /dev/null +++ b/util/combinations.go @@ -0,0 +1,41 @@ +package util + +// AllCombinationsUint32 returns all combinations of the input +func AllCombinationsUint32(parts [][]uint32) (out [][]uint32) { + + // allocate a slice to host all combinations + num := 1 + for _, part := range parts { + num *= len(part) + } + out = make([][]uint32, 0, num) + + // will contain idx of which one to pick for each part + indexes := make([]int, len(parts)) + +mainloop: + for { + // update indexes: + // travel backwards. whenever we encounter an index that "overflowed" + // reset it back to 0 and bump the previous one, until they are all maxed out + for i := len(indexes) - 1; i >= 0; i-- { + if indexes[i] >= len(parts[i]) { + if i == 0 { + break mainloop + } + indexes[i] = 0 + indexes[i-1]++ + } + } + + combo := make([]uint32, len(parts)) + for i, part := range parts { + combo[i] = part[indexes[i]] + } + out = append(out, combo) + + // always bump idx of the last one + indexes[len(parts)-1]++ + } + return out +} diff --git a/util/combinations_test.go b/util/combinations_test.go new file mode 100644 index 0000000000..debadc1933 --- /dev/null +++ b/util/combinations_test.go @@ -0,0 +1,71 @@ +package util + +import ( + "testing" + + "github.com/google/go-cmp/cmp" +) + +func TestAllCombinationsUint32(t *testing.T) { + type testCase struct { + in [][]uint32 + out [][]uint32 + } + testCases := []testCase{ + { + in: [][]uint32{ + {1, 10}, + }, + out: [][]uint32{ + {1}, + {10}, + }, + }, + { + in: [][]uint32{ + {1}, + {1, 10}, + {1}, + }, + out: [][]uint32{ + {1, 1, 1}, + {1, 10, 1}, + }, + }, + { + in: [][]uint32{ + {1, 2, 3}, + {1, 5, 10}, + {4, 8}, + }, + out: [][]uint32{ + {1, 1, 4}, + {1, 1, 8}, + {1, 5, 4}, + {1, 5, 8}, + {1, 10, 4}, + {1, 10, 8}, + {2, 1, 4}, + {2, 1, 8}, + {2, 5, 4}, + {2, 5, 8}, + {2, 10, 4}, + {2, 10, 8}, + {3, 1, 4}, + {3, 1, 8}, + {3, 5, 4}, + {3, 5, 8}, + {3, 10, 4}, + {3, 10, 8}, + }, + //4, 8, 20, 40, 40, 80, 8, 16, 40, 80, 80, 160, 12, 24, 60, 120, 120, 240}, + }, + } + + for i, tc := range testCases { + got := AllCombinationsUint32(tc.in) + if diff := cmp.Diff(tc.out, got); diff != "" { + t.Errorf("AllCombinationsUint32 test case %d mismatch (-want +got):\n%s", i, diff) + } + } +}