From 6e5f05005abba5767f063999a5e0f3c1a36b8bc6 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 2 Dec 2019 15:41:42 +0200 Subject: [PATCH 01/46] clarifications/simplification --- api/query_engine.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/api/query_engine.go b/api/query_engine.go index 7950336be0..18c69acb2c 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -25,6 +25,8 @@ var ( ) // alignRequests updates the requests with all details for fetching, making sure all metrics are in the same, optimal interval +// it chooses the highest resolution possible within ttl, but +// subjects the request to the max-points-per-req-{soft,hard} settings (lowering resolution to meet the soft setting) // note: it is assumed that all requests have the same maxDataPoints, 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) { @@ -132,7 +134,7 @@ func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint3 } 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 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 // divisible by the output interval) but let's not worry about that edge case. @@ -146,8 +148,10 @@ func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint3 pointsPerSerie := tsRange / interval if reqs[0].MaxPoints > 0 && pointsPerSerie > reqs[0].MaxPoints { + // 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(pointsPerSerie, reqs[0].MaxPoints) - pointsPerSerie = pointsPerSerie / aggNum + pointsPerSerie /= aggNum } pointsReturn := uint32(len(reqs)) * pointsPerSerie From 29d86ce27e19f057be0d9311739f77bba2dd6021 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 2 Dec 2019 10:05:28 +0200 Subject: [PATCH 02/46] WIP: don't over-align + implement MDP-optimization, pre-normalization --- api/graphite.go | 24 +++-- api/graphite_req.go | 46 ++++++++ api/query_engine.go | 188 ++++++++++++++++++++++----------- devdocs/expr.md | 25 +++++ devdocs/maxdatapoints.txt | 24 +++++ expr/func_aggregate.go | 2 + expr/func_aspercent.go | 17 +++ expr/func_countseries.go | 2 + expr/func_divideseries.go | 12 +++ expr/func_divideserieslists.go | 7 ++ expr/func_groupbytags.go | 1 + expr/func_smartsummarize.go | 2 + expr/func_summarize.go | 2 + expr/funcs.go | 10 +- expr/plan.go | 34 ++++-- 15 files changed, 309 insertions(+), 87 deletions(-) create mode 100644 api/graphite_req.go create mode 100644 devdocs/maxdatapoints.txt diff --git a/api/graphite.go b/api/graphite.go index c19fd4a155..896199c909 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -661,7 +661,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 +686,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)) } @@ -719,8 +719,8 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) } 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) + archive.Id, archive.NameWithTags(), r.Query, r.From, r.To, r.MDP, uint32(archive.Interval), cons, consReq, s.Node, archive.SchemaId, archive.AggId) + reqs.Add(newReq, r.PNGroup) } if tagquery.MetaTagSupport && len(metric.Defs) > 0 && len(metric.MetaTags) > 0 { @@ -739,31 +739,33 @@ 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 reqsList []models.Req + // TODO get rid of alignrequests and all "align" terminology + reqsList, meta.RenderStats.PointsFetch, meta.RenderStats.PointsReturn, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints) if err != nil { log.Errorf("HTTP Render alignReq error: %s", err.Error()) return nil, meta, err } 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 diff --git a/api/graphite_req.go b/api/graphite_req.go new file mode 100644 index 0000000000..0b917378a0 --- /dev/null +++ b/api/graphite_req.go @@ -0,0 +1,46 @@ +package api + +import ( + "fmt" + + "github.com/grafana/metrictank/api/models" + "github.com/grafana/metrictank/expr" +) + +// ReqMap is a map of requests of data, +// it has single requests for which no pre-normalization effort will be performed, and +// requests are that can be pre-normalized together to the same resolution, bundled by their PNGroup +type ReqMap struct { + single []models.Req + pngroups map[expr.PNGroup][]models.Req + cnt uint32 +} + +func NewReqMap() *ReqMap { + return &ReqMap{ + pngroups: make(map[expr.PNGroup][]models.Req), + } +} + +func (r *ReqMap) Add(req models.Req, group expr.PNGroup) { + r.cnt++ + if group == 0 { + r.single = append(r.single, req) + } + r.pngroups[group] = append(r.pngroups[group], req) +} +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 +} diff --git a/api/query_engine.go b/api/query_engine.go index 18c69acb2c..73b892794e 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -1,10 +1,11 @@ package api import ( - "math" + "fmt" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/api/response" + "github.com/grafana/metrictank/conf" "github.com/grafana/metrictank/consolidation" "github.com/grafana/metrictank/mdata" "github.com/grafana/metrictank/stats" @@ -24,80 +25,142 @@ var ( 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.") ) -// alignRequests updates the requests with all details for fetching, making sure all metrics are in the same, optimal interval -// it chooses the highest resolution possible within ttl, but -// subjects the request to the max-points-per-req-{soft,hard} settings (lowering resolution to meet the soft setting) -// note: it is assumed that all requests have the same maxDataPoints, from & to. +// planRequests updates the requests with all details for fetching. +// starting point: +// if MDP-optimization enabled -> select the coarsest archive that results in pointcount >= MDP/2, which hopefully lets us avoid, or at least minimize runtime consolidation. +// otherwise, the highest resolution possible within ttl +// then: +// * subjects the request to max-points-per-req-soft settings, lowering resolution to meet the soft setting. +// * subjects to max-points-per-req-hard setting, rejecting queries that can't be met +// * pre-normalization: a clue that series will need to be normalized together, so may as well try to read from archives that are equivalent to normalization +// need to watch out here, due to MDP-optimization might result in too low res here +// +// 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 - - var listIntervals []uint32 - var seenIntervals = make(map[uint32]struct{}) - var targets = make(map[string]struct{}) - - for _, req := range reqs { - targets[req.Target] = struct{}{} +func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) ([]models.Req, uint32, uint32, error) { + + var singleRets [][]conf.Retention + for i, req := range reqs.single { + rets := mdata.Schemas.Get(req.SchemaId).Retentions.Rets + singleRets = append(singleRets, rets) + var ok bool + if req.MaxPoints == 0 { + req, ok = initialHighestResWithinTTL(now, from, to, req, rets) + } else { + req, ok = initialLowestResForMDP(now, from, to, req, rets) + } + reqs.single[i] = req + if !ok { + return nil, 0, 0, errUnSatisfiable + } } - numTargets := uint32(len(targets)) - minTTL := now - reqs[0].From - minIntervalSoft := uint32(0) - minIntervalHard := uint32(0) + for group, groupReqs := range reqs.pngroups { + fmt.Println("processing group", group) + var groupRets [][]conf.Retention + for i, req := range groupReqs { + rets := mdata.Schemas.Get(req.SchemaId).Retentions.Rets + groupRets = append(singleRets, rets) + var ok bool + if req.MaxPoints == 0 { + req, ok = initialHighestResWithinTTL(now, from, to, req, rets) + } else { + req, ok = initialLowestResForMDP(now, from, to, req, rets) + } + reqs.pngroups[group][i] = req + if !ok { + return nil, 0, 0, errUnSatisfiable + } + } + reqs, _, _, _ := alignRequests(now, from, to, groupReqs) - if maxPointsPerReqSoft > 0 { - minIntervalSoft = uint32(math.Ceil(float64(tsRange) / (float64(maxPointsPerReqSoft) / float64(numTargets)))) - } - if maxPointsPerReqHard > 0 { - minIntervalHard = uint32(math.Ceil(float64(tsRange) / (float64(maxPointsPerReqHard) / float64(numTargets)))) } - // 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 { - // 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) - } + pointsReturn := uint32(len(reqs)) * pointsPerSerie + reqRenderPointsFetched.ValueUint32(pointsFetch) + reqRenderPointsReturned.ValueUint32(pointsReturn) - if req.TTL >= minTTL && req.ArchInterval >= minIntervalSoft { - break - } + return reqs, pointsFetch, pointsReturn, nil +} + +func initialHighestResWithinTTL(now, from, to uint32, req models.Req, rets []conf.Retention) (models.Req, bool) { + minTTL := now - from + var ok bool + for i, ret := range rets { + // skip non-ready option. + if ret.Ready > from { + continue } - if !found { - return nil, 0, 0, errUnSatisfiable + ok = 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) } + req.OutInterval = req.ArchInterval + req.AggNum = 1 + + if req.TTL >= minTTL { + break + } + } + return req, ok +} - if _, ok := seenIntervals[req.ArchInterval]; !ok { - listIntervals = append(listIntervals, req.ArchInterval) - seenIntervals[req.ArchInterval] = struct{}{} +func initialLowestResForMDP(now, from, to uint32, req models.Req, rets []conf.Retention) (models.Req, bool) { + tsRange := to - from + var ok bool + for i := len(rets) - 1; i >= 0; i-- { + ret := rets[i] + // skip non-ready option. + if ret.Ready > from { + continue + } + ok = 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) + } + req.OutInterval = req.ArchInterval + req.AggNum = 1 + + numPoints := tsRange / req.ArchInterval + if numPoints >= req.MaxPoints/2 { + // if numPoints > req.MaxPoints, we may be able to set up normalization here + // however, lets leave non-normalized for now. maybe this function will be used for PNGroups later. + // we wouldn't want to set up AggNum=2 if we need AggNum=3 for the LCM of the PNGroup + // why? imagine this scenario: + // interval 10s, numPoints 1000, req.MaxPoints 800 -> if we set AggNum to 2 now, we change interval to 20s + // but if we have a PNGroup with another series that has interval 30s, we would bring everything to 60s, needlessly coarse + // (we should bring everything to 30s instead) + // TODO AggNum/OutInterval not correct here + break } } + return req, ok +} +func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint32, uint32, error) { + + var listIntervals []uint32 + var seenIntervals = make(map[uint32]struct{}) + var targets = make(map[string]struct{}) + + for _, req := range reqs { + targets[req.Target] = 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 - } + // ??? NOW WHAT TODO RESUME HERE :) :{) // 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 @@ -147,16 +210,13 @@ func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint3 } pointsPerSerie := tsRange / interval - if reqs[0].MaxPoints > 0 && pointsPerSerie > reqs[0].MaxPoints { + + // TODO series are not same resolution, need to account for separate intervals + if planMDP > 0 && pointsPerSerie > 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(pointsPerSerie, reqs[0].MaxPoints) pointsPerSerie /= aggNum } - pointsReturn := uint32(len(reqs)) * pointsPerSerie - reqRenderPointsFetched.ValueUint32(pointsFetch) - reqRenderPointsReturned.ValueUint32(pointsReturn) - - return reqs, pointsFetch, pointsReturn, nil } diff --git a/devdocs/expr.md b/devdocs/expr.md index ef0b842633..3281701b9f 100644 --- a/devdocs/expr.md +++ b/devdocs/expr.md @@ -1,3 +1,28 @@ +Considerations when writing function: +make sure to return pointer, so that properties can be set, and 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 transparant or opaque aggregation. because those require special options. see https://github.com/grafana/metrictank/issues/926#issuecomment-559596384 +make sure to do the right things wrt getting slicepools, adding them to the cache for cleanup, not modifying tags, etc. (see below re memory management) + + +## MDP-optimization + +MDP at the leaf of the expr tree (fetch request) 0 means don't optimize, set it to >0 means, 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 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 to 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: transparant aggregations set PNGroups to the pointer value of that function, to uniquely identify any further data requests that will be fed into the same transparant aggregation. +* as we traverse down, any opaque aggregation functions and IA-functions reset PNGroup back to 0. + ## Management of point slices The `models.Series` type, even when passed by value, has a few fields that need special attention: diff --git a/devdocs/maxdatapoints.txt b/devdocs/maxdatapoints.txt new file mode 100644 index 0000000000..58c55a1e00 --- /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 alignRequests() for reporting diff --git a/expr/func_aggregate.go b/expr/func_aggregate.go index 91f11d802f..f913e168d7 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 = PNGroup(uintptr(unsafe.Pointer(s))) return context } diff --git a/expr/func_aspercent.go b/expr/func_aspercent.go index fb5780b70e..d794ff050d 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,22 @@ 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 + // 3) nodes-nil, total nil (and not a float) -> align all together + // 4) nodes-non-nil, total nil -> divides groupwise + // 5) nodes non-nil, total serieslist -> divides groupwise + + // note: we can't tell the difference between 1/2 up front, so we play it safe and don't align up front + // the only scenario where PNGroup is safe, is case 3 + if s.totalSeries == nil && s.nodes == nil { + context.PNGroup = models.PNGroup(uintptr(unsafe.Pointer(s))) + } else { + context.PNGroup = 0 + } return context } 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..189b1896d8 100644 --- a/expr/func_divideseries.go +++ b/expr/func_divideseries.go @@ -26,6 +26,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 } diff --git a/expr/func_divideserieslists.go b/expr/func_divideserieslists.go index eb39fecd1e..429053196b 100644 --- a/expr/func_divideserieslists.go +++ b/expr/func_divideserieslists.go @@ -26,6 +26,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 } diff --git a/expr/func_groupbytags.go b/expr/func_groupbytags.go index 7a5f20c1cc..594ebcabba 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 } 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..63255a5dcb 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 } diff --git a/expr/funcs.go b/expr/funcs.go index 0e986c75d8..6d50d9e49c 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -7,9 +7,11 @@ 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 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 } // GraphiteFunc defines a graphite processing function @@ -54,7 +56,7 @@ func init() { "aliasByTags": {NewAliasByNode, true}, "aliasByNode": {NewAliasByNode, true}, "aliasSub": {NewAliasSub, true}, - "asPercent": {NewAsPercent, true}, + "asPercent": {NewAsPercent, false}, // disabled because it needs an update for runtime normalization and possibly MDP/pre-normalization "avg": {NewAggregateConstructor("average", crossSeriesAvg), true}, "averageAbove": {NewFilterSeriesConstructor("average", ">"), true}, "averageBelow": {NewFilterSeriesConstructor("average", "<="), true}, diff --git a/expr/plan.go b/expr/plan.go index dd54727419..95a44d8d1a 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -11,12 +11,17 @@ import ( // 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 + PNGroup PNGroup + Cons consolidation.Consolidator // can be 0 to mean undefined + MDP uint32 // if we can MDP-optimize, reflects runtime consolidation MaxDataPoints. 0 otherwise. } +// PNGroup is an identifier for a pre-normalization group: data that can be pre-normalized together +type PNGroup uint64 + // 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 { @@ -28,6 +33,17 @@ func NewReq(query string, from, to uint32, cons consolidation.Consolidator) Req } } +func NewReqFromContext(query string, c Context) Req { + return Req{ + Query: query, + From: c.from, + To: c.to, + Cons: c.consol, + PNGroup: c.PNGroup, + MDP: c.MDP, + } +} + type Plan struct { Reqs []Req // data that needs to be fetched before functions can be executed funcs []GraphiteFunc // top-level funcs to execute, the head of each tree for each target @@ -69,8 +85,10 @@ func NewPlan(exprs []*expr, from, to, mdp uint32, stable bool, reqs []Req) (Plan 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 } fn, reqs, err = newplan(e, context, stable, reqs) if err != nil { @@ -94,7 +112,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 +122,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 } From 752cd66fb2f8218ad3293de6ca93f54d5ac9fdfb Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 2 Dec 2019 15:35:49 +0200 Subject: [PATCH 03/46] simplify NewPlan() --- api/graphite.go | 4 ++-- cmd/mt-explain/main.go | 2 +- expr/plan.go | 25 +++++++++++-------------- 3 files changed, 14 insertions(+), 17 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index 896199c909..ca872270af 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -225,7 +225,7 @@ 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) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { if request.NoProxy { @@ -1417,7 +1417,7 @@ 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) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable) if err != nil { response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) return diff --git a/cmd/mt-explain/main.go b/cmd/mt-explain/main.go index f3da607fa6..f80ff13e8c 100644 --- a/cmd/mt-explain/main.go +++ b/cmd/mt-explain/main.go @@ -76,7 +76,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) 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/expr/plan.go b/expr/plan.go index 95a44d8d1a..183ca42d88 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -79,31 +79,28 @@ 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) (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, MDP: mdp, PNGroup: 0, // making this explicit here for easy code grepping } - 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 From 41bca10219dadc5d5a65b3719610c1afb218d79a Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 2 Dec 2019 19:42:47 +0200 Subject: [PATCH 04/46] docs --- devdocs/alignrequests-too-course-grained.txt | 91 ++++++++++++++++++++ 1 file changed, 91 insertions(+) create mode 100644 devdocs/alignrequests-too-course-grained.txt diff --git a/devdocs/alignrequests-too-course-grained.txt b/devdocs/alignrequests-too-course-grained.txt new file mode 100644 index 0000000000..4c8f49f35f --- /dev/null +++ b/devdocs/alignrequests-too-course-grained.txt @@ -0,0 +1,91 @@ +## MDP-optimizable + +### Greedy-resolution functions + +A Greedy-resolution function (GR-function) is a certain processing function that requires, or may require, high resolution data 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 MDP-optimizable if we know for sure that the render request does not execute 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. + +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. + +This is a more refined solution of https://github.com/grafana/metrictank/issues/463. +In the past, we MDP-optimized every request, which led to incorrect data when fed into GR-functions. +We corrected it by turning off all MDP-optimizations, which I think led to increased latencies, though we don't have the stats off-hand. +The hope is by re-introducing MDP-optimizations the correct way, we can speed up many requests again. + +## Pre-normalizable + +### 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 simplicty, we don't implement this. + +### Transparent aggregation + +A trans-aggregation is a processing function that aggregates multiple series together in a predictable way (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 + +when data will be used together (e.g. aggregating multiple series together) they will need to have the same resolution. +(note that generally, series do *not* need to have the same resolution. We have been aligning resolutions much too aggressively. see https://github.com/grafana/metrictank/issues/926) +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) +This is the case when series go from fetching to transparent aggregation, possibly with some processing functions - except opaque aggregation(s) or IA-function(s) - in between. + +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. + + +## Proposed changes + +1) Don't align any requests (as in `models.Req`) up front +2) Make sure all aggregation functions can normalize at runtime (if necessary), otherwise they will fail to process multiple input series that now can have different intervals +3) Implement pre-normalization +4) While we're at it, may as well implement MDP-optimization +5) (planning-stage awareness of the output interval of IA-functions, which means we can set up fetching/(pre)normalization in a smarter way) + +Step 1 and 2 will solve our most urgent problem of over-aligning data (https://github.com/grafana/metrictank/issues/926) +However it will probably (?) leave some performance optimizations on the table. Which step 3 and 4 address. It's unclear how urgent step 3 and 4 are, though they aren't too difficult to implement. +Implementing both of them can probably be done in one shot, as solving them is done in a similar way. + +Note that step 1 has a complication: +since we no longer set up all models.Req for a request at once, it's trickier to implement "max-points-per-req-soft" and "max-points-per-req-hard". +Think of it this way: if the number of datapoints fetched in a request is higher than the soft limit, which series should we fetch in a more course way? We can implement a heuristic that picks series that are highest resolution, and fetch the next rollup for them, trying to avoid those series that will be processed by a GR-function - though if we have no other choice, we will do it, as we currently do. We keep making fetches coarser until we fetch the coarsest archives of all series. Then we compare against the hard limit and bail out if the limit is breached. Similar to how we currently do it. +Note that to keep the support for the max-points-per-req-soft/max-points-per-req-hard setting we have to implement tracking of GR-functions which means we can probably just do step 4 while we're at it. + +Step 5 is something that can wait. From d1fb28a4ec13fac5ebe1b91e19731ba6fdf18824 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 2 Dec 2019 21:22:30 +0200 Subject: [PATCH 05/46] aggregators now need to be able to normalize at runtime --- expr/func_aggregate.go | 1 + expr/func_aspercent.go | 28 ++++++++-------- expr/func_divideseries.go | 23 +++++++++++++- expr/func_divideserieslists.go | 6 ++-- expr/func_groupbytags.go | 2 +- expr/normalize.go | 58 ++++++++++++++++++++++++++++++++++ 6 files changed, 101 insertions(+), 17 deletions(-) create mode 100644 expr/normalize.go diff --git a/expr/func_aggregate.go b/expr/func_aggregate.go index f913e168d7..909809db8b 100644 --- a/expr/func_aggregate.go +++ b/expr/func_aggregate.go @@ -48,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_aspercent.go b/expr/func_aspercent.go index d794ff050d..99eb0519ea 100644 --- a/expr/func_aspercent.go +++ b/expr/func_aspercent.go @@ -152,8 +152,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} @@ -179,7 +179,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) @@ -207,19 +207,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) @@ -262,7 +264,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] } diff --git a/expr/func_divideseries.go b/expr/func_divideseries.go index 189b1896d8..da54206873 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 @@ -53,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 the require no 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, diff --git a/expr/func_divideserieslists.go b/expr/func_divideserieslists.go index 429053196b..7c9ef41fc8 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 @@ -50,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{ diff --git a/expr/func_groupbytags.go b/expr/func_groupbytags.go index 594ebcabba..f6f44b7547 100644 --- a/expr/func_groupbytags.go +++ b/expr/func_groupbytags.go @@ -130,7 +130,7 @@ func (s *FuncGroupByTags) Exec(cache map[Req][]models.Series) ([]models.Series, 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/normalize.go b/expr/normalize.go new file mode 100644 index 0000000000..258d1d88b3 --- /dev/null +++ b/expr/normalize.go @@ -0,0 +1,58 @@ +package expr + +import ( + "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 + // TODO verify that Consolidate()'s behavior around odd-sized inputs makes sense. reread old stuff re canonical form etc + datapoints := pointSlicePool.Get().([]schema.Point) + datapoints = append(datapoints, in.Datapoints...) + in.Datapoints = consolidation.Consolidate(datapoints, interval/in.Interval, in.Consolidator) + in.Interval = interval / in.Interval + cache[Req{}] = append(cache[Req{}], in) + return in +} From 6b5c9912edde5bcf11efaadf7e90008761abed9e Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Thu, 16 Jan 2020 23:12:10 +0100 Subject: [PATCH 06/46] helper to generate combinations of uint32's --- util/combinations.go | 41 ++++++++++++++++++++++ util/combinations_test.go | 71 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 112 insertions(+) create mode 100644 util/combinations.go create mode 100644 util/combinations_test.go diff --git a/util/combinations.go b/util/combinations.go new file mode 100644 index 0000000000..88a225d3d6 --- /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 + idexes := make([]int, len(parts)) + +mainloop: + for { + // update idexes: + // 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(idexes) - 1; i >= 0; i-- { + if idexes[i] >= len(parts[i]) { + if i == 0 { + break mainloop + } + idexes[i] = 0 + idexes[i-1]++ + } + } + + combo := make([]uint32, len(parts)) + for i, part := range parts { + combo[i] = part[idexes[i]] + } + out = append(out, combo) + + // always bump idx of the last one + idexes[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) + } + } +} From 89ad2108783e9d6b228c17a797e54845376d05f1 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 15 Jan 2020 21:55:45 +0100 Subject: [PATCH 07/46] rework request planning * rethink alignRequests heuristic * cleanup code for DRY --- api/graphite.go | 11 +- api/graphite_req.go | 114 +++++++++ api/models/request.go | 73 +++++- api/query_engine.go | 385 +++++++++++++++++++---------- api/query_engine_test.go | 3 + devdocs/expr.md | 5 +- devdocs/maxdatapoints.txt | 2 +- devdocs/render-request-handling.md | 4 +- docs/consolidation.md | 2 +- docs/http-api.md | 2 +- docs/metrics.md | 3 +- docs/render-path.md | 2 +- 12 files changed, 457 insertions(+), 149 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index ca872270af..e1b104aa12 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -748,11 +748,14 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) // 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 - var reqsList []models.Req - // TODO get rid of alignrequests and all "align" terminology - reqsList, meta.RenderStats.PointsFetch, meta.RenderStats.PointsReturn, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints) + var rp *ReqsPlan + rp, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints) + meta.RenderStats.PointsFetch = rp.PointsFetch() + meta.RenderStats.PointsReturn = rp.PointsReturn(plan.MaxDataPoints) + reqsList := rp.List() + if err != nil { - log.Errorf("HTTP Render alignReq error: %s", err.Error()) + log.Errorf("HTTP Render planRequests error: %s", err.Error()) return nil, meta, err } span := opentracing.SpanFromContext(ctx) diff --git a/api/graphite_req.go b/api/graphite_req.go index 0b917378a0..bb72821a87 100644 --- a/api/graphite_req.go +++ b/api/graphite_req.go @@ -44,3 +44,117 @@ func (r ReqMap) Dump() string { } 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 +type ReqsPlan struct { + pngroups map[expr.PNGroup]PNGroupSplit + single PNGroupSplit + cnt uint32 +} + +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 +} + +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 +} + +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 +} + +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/request.go b/api/models/request.go index 997d7e42e6..07bb5b67f6 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -3,6 +3,7 @@ package models import ( "fmt" + "github.com/grafana/metrictank/conf" "github.com/grafana/metrictank/schema" "github.com/grafana/metrictank/cluster" @@ -22,7 +23,7 @@ type Req struct { To uint32 `json:"to"` MaxPoints uint32 `json:"maxPoints"` 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 +36,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,7 +44,7 @@ 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 +// NewReq creates a new request. It sets all properties minus the ones that need request planning func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterval uint32, cons, consReq consolidation.Consolidator, node cluster.Node, schemaId, aggId uint16) Req { return Req{ MKey: key, @@ -61,6 +62,70 @@ func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterv } } +// 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 +} + +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 + // divisible by the output interval) but let's not worry about that edge case. + r.PlanNormalization(interval) +} + +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) } @@ -98,7 +163,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) +// * TTL (because planRequests 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 diff --git a/api/query_engine.go b/api/query_engine.go index 73b892794e..3089d6919a 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -1,12 +1,12 @@ package api import ( - "fmt" + "math" + "reflect" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/api/response" "github.com/grafana/metrictank/conf" - "github.com/grafana/metrictank/consolidation" "github.com/grafana/metrictank/mdata" "github.com/grafana/metrictank/stats" "github.com/grafana/metrictank/util" @@ -18,72 +18,155 @@ 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.") ) +func getRetentions(req models.Req) []conf.Retention { + return mdata.Schemas.Get(req.SchemaId).Retentions.Rets +} + // planRequests updates the requests with all details for fetching. -// starting point: -// if MDP-optimization enabled -> select the coarsest archive that results in pointcount >= MDP/2, which hopefully lets us avoid, or at least minimize runtime consolidation. -// otherwise, the highest resolution possible within ttl -// then: -// * subjects the request to max-points-per-req-soft settings, lowering resolution to meet the soft setting. -// * subjects to max-points-per-req-hard setting, rejecting queries that can't be met -// * pre-normalization: a clue that series will need to be normalized together, so may as well try to read from archives that are equivalent to normalization -// need to watch out here, due to MDP-optimization might result in too low res here +// 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 planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) ([]models.Req, uint32, uint32, error) { +func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan, error) { - var singleRets [][]conf.Retention - for i, req := range reqs.single { - rets := mdata.Schemas.Get(req.SchemaId).Retentions.Rets - singleRets = append(singleRets, rets) - var ok bool - if req.MaxPoints == 0 { - req, ok = initialHighestResWithinTTL(now, from, to, req, rets) - } else { - req, ok = initialLowestResForMDP(now, from, to, req, rets) + ok, rp := false, NewReqsPlan(*reqs) + + 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 } - reqs.single[i] = req + } + for i, req := range rp.single.mdpno { + rp.single.mdpno[i], ok = planHighestResSingle(now, from, to, req) if !ok { - return nil, 0, 0, errUnSatisfiable + 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 } } - for group, groupReqs := range reqs.pngroups { - fmt.Println("processing group", group) - var groupRets [][]conf.Retention - for i, req := range groupReqs { - rets := mdata.Schemas.Get(req.SchemaId).Retentions.Rets - groupRets = append(singleRets, rets) - var ok bool - if req.MaxPoints == 0 { - req, ok = initialHighestResWithinTTL(now, from, to, req, rets) - } else { - req, ok = initialLowestResForMDP(now, from, to, req, rets) + if maxPointsPerReqSoft > 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(maxPointsPerReqSoft) { + 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(maxPointsPerReqSoft) { + goto HonoredSoft + } + } } - reqs.pngroups[group][i] = req - if !ok { - return nil, 0, 0, errUnSatisfiable + 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(maxPointsPerReqSoft) { + goto HonoredSoft + } + } } } - reqs, _, _, _ := alignRequests(now, from, to, groupReqs) + } +HonoredSoft: + + if maxPointsPerReqHard > 0 && int(rp.PointsFetch()) > maxPointsPerReqHard { + return nil, errMaxPointsPerReq } - pointsReturn := uint32(len(reqs)) * pointsPerSerie - reqRenderPointsFetched.ValueUint32(pointsFetch) - reqRenderPointsReturned.ValueUint32(pointsReturn) + // 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)) + } + } + reqRenderPointsFetched.ValueUint32(rp.PointsFetch()) + reqRenderPointsReturned.ValueUint32(rp.PointsReturn(planMDP)) - return reqs, pointsFetch, pointsReturn, nil + return &rp, nil } -func initialHighestResWithinTTL(now, from, to uint32, req models.Req, rets []conf.Retention) (models.Req, bool) { +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 { @@ -92,16 +175,7 @@ func initialHighestResWithinTTL(now, from, to uint32, req models.Req, rets []con continue } ok = 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) - } - req.OutInterval = req.ArchInterval - req.AggNum = 1 + req.Plan(i, ret) if req.TTL >= minTTL { break @@ -110,113 +184,160 @@ func initialHighestResWithinTTL(now, from, to uint32, req models.Req, rets []con return req, ok } -func initialLowestResForMDP(now, from, to uint32, req models.Req, rets []conf.Retention) (models.Req, bool) { - tsRange := to - from +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-- { - ret := rets[i] // skip non-ready option. - if ret.Ready > from { + if rets[i].Ready > from { continue } ok = 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) - } - req.OutInterval = req.ArchInterval - req.AggNum = 1 - - numPoints := tsRange / req.ArchInterval - if numPoints >= req.MaxPoints/2 { - // if numPoints > req.MaxPoints, we may be able to set up normalization here - // however, lets leave non-normalized for now. maybe this function will be used for PNGroups later. - // we wouldn't want to set up AggNum=2 if we need AggNum=3 for the LCM of the PNGroup - // why? imagine this scenario: - // interval 10s, numPoints 1000, req.MaxPoints 800 -> if we set AggNum to 2 now, we change interval to 20s - // but if we have a PNGroup with another series that has interval 30s, we would bring everything to 60s, needlessly coarse - // (we should bring everything to 30s instead) - // TODO AggNum/OutInterval not correct here + req.Plan(i, rets[i]) + if req.PointsFetch() >= req.MaxPoints/2 { break } } return req, ok } -func alignRequests(now, from, to uint32, reqs []models.Req) ([]models.Req, uint32, uint32, error) { +func planHighestResMulti(now, from, to uint32, reqs []models.Req) ([]models.Req, bool) { + minTTL := now - from var listIntervals []uint32 var seenIntervals = make(map[uint32]struct{}) - var targets = make(map[string]struct{}) - for _, req := range reqs { - targets[req.Target] = struct{}{} - } + for i := range reqs { + req := &reqs[i] + var ok bool + rets := getRetentions(*req) + for i, ret := range rets { + // skip non-ready option. + if ret.Ready > from { + continue + } + ok = true + req.Plan(i, ret) - // 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 + if req.TTL >= minTTL { + break + } + } + if !ok { + return nil, ok + } + if _, ok := seenIntervals[req.ArchInterval]; !ok { + listIntervals = append(listIntervals, req.ArchInterval) + seenIntervals[req.ArchInterval] = struct{}{} + } + } interval := util.Lcm(listIntervals) - // ??? NOW WHAT TODO RESUME HERE :) :{) + // 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)) + } + + return reqs, true +} + +// 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 - // 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) + // 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 - 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 - } - + var ok bool + rets := getRetentions(*req) + var validIntervals []uint32 + for _, ret := range rets { + if ret.Ready <= from && req.TTL >= 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 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 - // 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 + combos := util.AllCombinationsUint32(validIntervalss) + var maxScore int - // TODO series are not same resolution, need to account for separate intervals - if planMDP > 0 && pointsPerSerie > 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(pointsPerSerie, reqs[0].MaxPoints) - pointsPerSerie /= aggNum + 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, ok } diff --git a/api/query_engine_test.go b/api/query_engine_test.go index 9afd6363d5..729b688306 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -16,6 +16,7 @@ import ( func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { var schemas []conf.Schema oriMaxPointsPerReqSoft := maxPointsPerReqSoft + oriMaxPointsPerHardReq := maxPointsPerReqHard for _, ret := range retentions { schemas = append(schemas, conf.Schema{ @@ -28,6 +29,7 @@ func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models maxPointsPerReqSoft = points } } + maxPointsPerReqHard = maxPointsPerReqSoft * 10 mdata.Schemas = conf.NewSchemas(schemas) out, _, _, err := alignRequests(now, reqs[0].From, reqs[0].To, reqs) @@ -45,6 +47,7 @@ func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models } maxPointsPerReqSoft = oriMaxPointsPerReqSoft + maxPointsPerReqHard = oriMaxPointsPerHardReq } // 2 series requested with equal raw intervals. req 0-30. now 1200. one archive of ttl=1200 does it diff --git a/devdocs/expr.md b/devdocs/expr.md index 3281701b9f..7fd16c07aa 100644 --- a/devdocs/expr.md +++ b/devdocs/expr.md @@ -21,7 +21,8 @@ However, multiple series getting fetched that then get aggregated together, may 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: transparant aggregations set PNGroups to the pointer value of that function, to uniquely identify any further data requests that will be fed into the same transparant aggregation. -* as we traverse down, any opaque aggregation functions and IA-functions reset PNGroup back to 0. +* 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. ## Management of point slices @@ -125,7 +126,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. diff --git a/devdocs/maxdatapoints.txt b/devdocs/maxdatapoints.txt index 58c55a1e00..a3997733d6 100644 --- a/devdocs/maxdatapoints.txt +++ b/devdocs/maxdatapoints.txt @@ -21,4 +21,4 @@ mdp set from GET param, but 0 if came from graphite -> executePlan() models.NewReq() -> models.Req.MaxPoints -> planRequests(): used for MDP-optimization -> plan.MaxDatapoints used for final runtime consolidation - -> and also used in alignRequests() for reporting + -> and also used in planRequests() for reporting diff --git a/devdocs/render-request-handling.md b/devdocs/render-request-handling.md index a694491973..5c31499f8f 100644 --- a/devdocs/render-request-handling.md +++ b/devdocs/render-request-handling.md @@ -12,8 +12,8 @@ * 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 at all models.Req objects, which means decide 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. 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..5c8cbbaf82 100644 --- a/docs/http-api.md +++ b/docs/http-api.md @@ -234,7 +234,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..32ac15823e 100644 --- a/docs/render-path.md +++ b/docs/render-path.md @@ -89,6 +89,6 @@ such that they can be used together (for aggregating, merging, etc) TODO talk about -alignRequests -> getTargets -> mergeSeries -> sort Series -> plan.Run (executes functions and does MDP consolidation with nudging) +planRequests -> getTargets -> mergeSeries -> sort Series -> plan.Run (executes functions and does MDP consolidation with nudging) talk more about what happens at each step, how data is manipulated etc From f0f9b1710bfcea95faa2e4df844b62e8c580efbe Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Thu, 16 Jan 2020 15:31:18 +0100 Subject: [PATCH 08/46] Loop new request identifiers/differentiators through request path if multiple queries are given (e.g. target=foo*&target=sum(foo*)&target=summarize(foo*)) we need to be able to differentiate them. We wouldn't want to conflate MDP or PN-optimized data when we need as high resolution data as possible, for example. We already differentiate expr.Req, models.Req and models.Series based on query, from, to, consolidator, but now we need to do it by the requests' MDP and PNGroup as well. Note that FuncGet() is setup with an expr.Req early on in the request parsing phase (NewPlan()), whereas the data it will load later on first gets turned into a models.Req in executePlan() and then in models.Series in getTargets(). Then from those series we parse out again what the original expr.Req was and classify the data by expr.Req when we issue plan.Run(), such that FuncGet() is able to load the data based on the expr.Req it has. Note also that models.Req and models.Series may be transmitted over the network. We als update mergeSeries accordingly to not conflate optimized and non-optimized data for the same metric together. (see query example above). models.Req : - add PNGroup - be more explicit that we take values from expr.Req - we can't import expr, so: * refactor how we create it (we can't take in expr.Req as an arg * move PNGroup from expr to models models.Series: - add MDP and PNGroup --- api/dataprocessor.go | 18 ++++++---- api/graphite.go | 8 ++--- api/graphite_req.go | 14 ++++---- api/models/request.go | 39 +++++++++++---------- api/models/series.go | 4 +++ api/models/series_gen.go | 64 +++++++++++++++++++++++++++++++--- expr/func_aggregate.go | 2 +- expr/func_aspercent.go | 2 ++ expr/func_divideseries.go | 2 ++ expr/func_divideserieslists.go | 2 ++ expr/func_groupbytags.go | 2 ++ expr/func_persecond.go | 14 ++++---- expr/func_summarize.go | 18 +++++----- expr/funcs.go | 2 +- expr/plan.go | 33 +++++++++++++++--- 15 files changed, 163 insertions(+), 61 deletions(-) diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 06bffbbf91..8544da171f 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -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" @@ -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) } diff --git a/api/graphite.go b/api/graphite.go index e1b104aa12..5f5348d7f7 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -718,9 +718,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, r.MDP, uint32(archive.Interval), cons, consReq, s.Node, archive.SchemaId, archive.AggId) - reqs.Add(newReq, r.PNGroup) + newReq := r.ToModel() + newReq.Init(archive, cons, s.Node) + reqs.Add(newReq) } if tagquery.MetaTagSupport && len(metric.Defs) > 0 && len(metric.MetaTags) > 0 { @@ -792,7 +792,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) } diff --git a/api/graphite_req.go b/api/graphite_req.go index bb72821a87..02f2fd758c 100644 --- a/api/graphite_req.go +++ b/api/graphite_req.go @@ -4,7 +4,6 @@ import ( "fmt" "github.com/grafana/metrictank/api/models" - "github.com/grafana/metrictank/expr" ) // ReqMap is a map of requests of data, @@ -12,22 +11,23 @@ import ( // requests are that can be pre-normalized together to the same resolution, bundled by their PNGroup type ReqMap struct { single []models.Req - pngroups map[expr.PNGroup][]models.Req + pngroups map[models.PNGroup][]models.Req cnt uint32 } func NewReqMap() *ReqMap { return &ReqMap{ - pngroups: make(map[expr.PNGroup][]models.Req), + pngroups: make(map[models.PNGroup][]models.Req), } } -func (r *ReqMap) Add(req models.Req, group expr.PNGroup) { +func (r *ReqMap) Add(req models.Req) { r.cnt++ - if group == 0 { + if req.PNGroup == 0 { r.single = append(r.single, req) + return } - r.pngroups[group] = append(r.pngroups[group], req) + r.pngroups[req.PNGroup] = append(r.pngroups[req.PNGroup], req) } func (r ReqMap) Dump() string { out := fmt.Sprintf("ReqsMap (%d entries):\n", r.cnt) @@ -53,7 +53,7 @@ type PNGroupSplit struct { // ReqsPlan holds requests that have been planned type ReqsPlan struct { - pngroups map[expr.PNGroup]PNGroupSplit + pngroups map[models.PNGroup]PNGroupSplit single PNGroupSplit cnt uint32 } diff --git a/api/models/request.go b/api/models/request.go index 07bb5b67f6..0f8c895188 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -4,6 +4,7 @@ import ( "fmt" "github.com/grafana/metrictank/conf" + "github.com/grafana/metrictank/idx" "github.com/grafana/metrictank/schema" "github.com/grafana/metrictank/cluster" @@ -22,6 +23,7 @@ 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 (pre-normalization and runtime normalization). (but not runtime consolidation) // ConsReq 0 -> configured value @@ -44,22 +46,19 @@ 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 planning -func NewReq(key schema.MKey, target, patt string, from, to, maxPoints, rawInterval uint32, cons, consReq consolidation.Consolidator, node cluster.Node, schemaId, aggId uint16) Req { - return Req{ - MKey: key, - Target: target, - Pattern: patt, - From: from, - To: to, - MaxPoints: maxPoints, - RawInterval: rawInterval, - Consolidator: cons, - ConsReq: consReq, - Node: node, - SchemaId: schemaId, - AggId: aggId, - } +// PNGroup is an identifier for a pre-normalization group: data that can be pre-normalized together +type PNGroup uint64 + +// Init initializes a request based on the metadata that we know of. +// It sets all properties minus 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 @@ -131,8 +130,8 @@ func (r Req) String() string { } 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 @@ -148,6 +147,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()), @@ -188,6 +188,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 } diff --git a/api/models/series.go b/api/models/series.go index 59b7ebeafb..879a3d25e7 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 } @@ -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/expr/func_aggregate.go b/expr/func_aggregate.go index 909809db8b..9b9567d83e 100644 --- a/expr/func_aggregate.go +++ b/expr/func_aggregate.go @@ -27,7 +27,7 @@ func (s *FuncAggregate) Signature() ([]Arg, []Arg) { } func (s *FuncAggregate) Context(context Context) Context { - context.PNGroup = PNGroup(uintptr(unsafe.Pointer(s))) + context.PNGroup = models.PNGroup(uintptr(unsafe.Pointer(s))) return context } diff --git a/expr/func_aspercent.go b/expr/func_aspercent.go index 99eb0519ea..5bc3f28ec6 100644 --- a/expr/func_aspercent.go +++ b/expr/func_aspercent.go @@ -304,6 +304,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_divideseries.go b/expr/func_divideseries.go index da54206873..3d823a8bd4 100644 --- a/expr/func_divideseries.go +++ b/expr/func_divideseries.go @@ -102,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 7c9ef41fc8..398145f3aa 100644 --- a/expr/func_divideserieslists.go +++ b/expr/func_divideserieslists.go @@ -78,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 f6f44b7547..b5a4427cc6 100644 --- a/expr/func_groupbytags.go +++ b/expr/func_groupbytags.go @@ -125,6 +125,8 @@ 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() 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_summarize.go b/expr/func_summarize.go index 63255a5dcb..54f23c9081 100644 --- a/expr/func_summarize.go +++ b/expr/func_summarize.go @@ -70,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 6d50d9e49c..b9ce5cf63f 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -10,7 +10,7 @@ type Context struct { from uint32 to uint32 consol consolidation.Consolidator // can be 0 to mean undefined - PNGroup PNGroup // pre-normalization group. if the data can be safely pre-normalized + 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 } diff --git a/expr/plan.go b/expr/plan.go index 183ca42d88..955682be4d 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -14,14 +14,11 @@ 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 - PNGroup PNGroup Cons consolidation.Consolidator // can be 0 to mean undefined - MDP uint32 // if we can MDP-optimize, reflects runtime consolidation MaxDataPoints. 0 otherwise. + PNGroup models.PNGroup + MDP uint32 // if we can MDP-optimize, reflects runtime consolidation MaxDataPoints. 0 otherwise. } -// PNGroup is an identifier for a pre-normalization group: data that can be pre-normalized together -type PNGroup uint64 - // 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 { @@ -44,6 +41,32 @@ func NewReqFromContext(query string, c Context) Req { } } +// 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, + } +} + type Plan struct { Reqs []Req // data that needs to be fetched before functions can be executed funcs []GraphiteFunc // top-level funcs to execute, the head of each tree for each target From 5dd295e2657509c543a6657b7e8562671740b26a Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 17 Jan 2020 20:03:04 +0100 Subject: [PATCH 09/46] make fetch optimizations optional --- api/config.go | 2 ++ api/graphite.go | 4 +-- cmd/mt-explain/main.go | 3 +- docker/docker-chaos/metrictank.ini | 2 ++ docker/docker-cluster-query/metrictank.ini | 2 ++ docker/docker-cluster/metrictank.ini | 2 ++ .../metrictank.ini | 2 ++ docs/config.md | 2 ++ expr/funcs.go | 11 ++++---- expr/plan.go | 28 +++++++++++-------- metrictank-sample.ini | 2 ++ scripts/config/metrictank-docker.ini | 2 ++ scripts/config/metrictank-package.ini | 2 ++ 13 files changed, 44 insertions(+), 20 deletions(-) diff --git a/api/config.go b/api/config.go index f55b0dfb39..5f24689e1c 100644 --- a/api/config.go +++ b/api/config.go @@ -28,6 +28,7 @@ var ( getTargetsConcurrency int tagdbDefaultLimit uint speculationThreshold float64 + fetchOptimizations bool graphiteProxy *httputil.ReverseProxy timeZone *time.Location @@ -49,6 +50,7 @@ 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(&fetchOptimizations, "fetch-optimizations", false, "enable MaxDataPoints optimization and pre-normalization optimizations") globalconf.Register("http", apiCfg, flag.ExitOnError) } diff --git a/api/graphite.go b/api/graphite.go index 5f5348d7f7..2a7eb76231 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -225,7 +225,7 @@ 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) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, fetchOptimizations) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { if request.NoProxy { @@ -1420,7 +1420,7 @@ 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) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, fetchOptimizations) if err != nil { response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) return diff --git a/cmd/mt-explain/main.go b/cmd/mt-explain/main.go index f80ff13e8c..5ec1d6e8eb 100644 --- a/cmd/mt-explain/main.go +++ b/cmd/mt-explain/main.go @@ -25,6 +25,7 @@ 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)") + fetchOptimizations := flag.Bool("fetch-optimization", false, "enable MaxDataPoints optimization and pre-normalization optimizations") flag.Usage = func() { fmt.Println("mt-explain") @@ -76,7 +77,7 @@ func main() { return } - plan, err := expr.NewPlan(exps, fromUnix, toUnix, uint32(*mdp), *stable) + plan, err := expr.NewPlan(exps, fromUnix, toUnix, uint32(*mdp), *stable, *fetchOptimizations) 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/docker/docker-chaos/metrictank.ini b/docker/docker-chaos/metrictank.ini index 06c2026c12..88a81e7329 100644 --- a/docker/docker-chaos/metrictank.ini +++ b/docker/docker-chaos/metrictank.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## diff --git a/docker/docker-cluster-query/metrictank.ini b/docker/docker-cluster-query/metrictank.ini index f77a8e226c..a1ac41efe6 100644 --- a/docker/docker-cluster-query/metrictank.ini +++ b/docker/docker-cluster-query/metrictank.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## diff --git a/docker/docker-cluster/metrictank.ini b/docker/docker-cluster/metrictank.ini index 33e0526d36..350ae7038b 100644 --- a/docker/docker-cluster/metrictank.ini +++ b/docker/docker-cluster/metrictank.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = 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..7c29680fe4 100644 --- a/docker/docker-dev-custom-cfg-kafka/metrictank.ini +++ b/docker/docker-dev-custom-cfg-kafka/metrictank.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## diff --git a/docs/config.md b/docs/config.md index e95fbce6e6..402506d19e 100644 --- a/docs/config.md +++ b/docs/config.md @@ -293,6 +293,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ``` ## metric data inputs ## diff --git a/expr/funcs.go b/expr/funcs.go index b9ce5cf63f..ca95f0d23d 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -7,11 +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 - 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 + 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 + fetchOptimizations bool // disable PNGroups and MDP } // GraphiteFunc defines a graphite processing function diff --git a/expr/plan.go b/expr/plan.go index 955682be4d..8294b8c9cf 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -31,14 +31,17 @@ func NewReq(query string, from, to uint32, cons consolidation.Consolidator) Req } func NewReqFromContext(query string, c Context) Req { - return Req{ - Query: query, - From: c.from, - To: c.to, - Cons: c.consol, - PNGroup: c.PNGroup, - MDP: c.MDP, + r := Req{ + Query: query, + From: c.from, + To: c.to, + Cons: c.consol, + } + if c.fetchOptimizations { + r.PNGroup = c.PNGroup + r.MDP = c.MDP } + return r } // NewReqFromSeries generates a Req back from a series @@ -102,7 +105,7 @@ 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) (Plan, error) { +func NewPlan(exprs []*expr, from, to, mdp uint32, stable, fetchOptimizations bool) (Plan, error) { plan := Plan{ exprs: exprs, MaxDataPoints: mdp, @@ -111,10 +114,11 @@ func NewPlan(exprs []*expr, from, to, mdp uint32, stable bool) (Plan, error) { } for _, e := range exprs { context := Context{ - from: from, - to: to, - MDP: mdp, - PNGroup: 0, // making this explicit here for easy code grepping + from: from, + to: to, + MDP: mdp, + PNGroup: 0, // making this explicit here for easy code grepping + fetchOptimizations: fetchOptimizations, } fn, reqs, err := newplan(e, context, stable, plan.Reqs) if err != nil { diff --git a/metrictank-sample.ini b/metrictank-sample.ini index 00184eacce..ca7bb221b0 100644 --- a/metrictank-sample.ini +++ b/metrictank-sample.ini @@ -248,6 +248,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## diff --git a/scripts/config/metrictank-docker.ini b/scripts/config/metrictank-docker.ini index 0e383d3403..08ed99ed5b 100644 --- a/scripts/config/metrictank-docker.ini +++ b/scripts/config/metrictank-docker.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## diff --git a/scripts/config/metrictank-package.ini b/scripts/config/metrictank-package.ini index f72de06709..1b0c04b970 100644 --- a/scripts/config/metrictank-package.ini +++ b/scripts/config/metrictank-package.ini @@ -244,6 +244,8 @@ 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 MaxDataPoints optimization and pre-normalization optimizations +fetch-optimizations = false ## metric data inputs ## From 0f28be9dbbc94102dda34a34426be182be2214bf Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 17 Jan 2020 21:53:14 +0100 Subject: [PATCH 10/46] mt-explain: support new req information --- cmd/mt-explain/main.go | 1 + expr/plan.go | 14 +++++++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/cmd/mt-explain/main.go b/cmd/mt-explain/main.go index 5ec1d6e8eb..8c5fd099a0 100644 --- a/cmd/mt-explain/main.go +++ b/cmd/mt-explain/main.go @@ -33,6 +33,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") diff --git a/expr/plan.go b/expr/plan.go index 8294b8c9cf..24f0f15250 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -90,8 +90,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 { + 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.Fprintln(w, " ", r) + 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) From 531a5f5d5754787950e7af57557fdb82ce827922 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sat, 18 Jan 2020 17:56:58 +0100 Subject: [PATCH 11/46] clean up expr docs. * move MDP and PN-optimization specifics to render-path docs * make expr devdocs easier to understand --- devdocs/expr.md | 72 ++++++++++++++++++++++----------------------- docs/render-path.md | 20 +++++++++++++ 2 files changed, 55 insertions(+), 37 deletions(-) diff --git a/devdocs/expr.md b/devdocs/expr.md index 7fd16c07aa..093320e032 100644 --- a/devdocs/expr.md +++ b/devdocs/expr.md @@ -1,30 +1,37 @@ -Considerations when writing function: -make sure to return pointer, so that properties can be set, and we get a consistent PNGroup through the pipeline (if applicable) +# 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 transparant or opaque aggregation. because those require special options. see https://github.com/grafana/metrictank/issues/926#issuecomment-559596384 -make sure to do the right things wrt getting slicepools, adding them to the cache for cleanup, not modifying tags, etc. (see below re memory management) +## consider whether the function is GR, IA, a transparent or opaque aggregation, or combine different series together somehow -## MDP-optimization +Such functions require special options. +see https://github.com/grafana/metrictank/issues/926#issuecomment-559596384 -MDP at the leaf of the expr tree (fetch request) 0 means don't optimize, set it to >0 means, 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 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 to MDP to 0 on the context (and thus also on any subsequent requests) +## implement our copy-o-write approach when dealing with modifying series -## Pre-normalization +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 that during plan cleanup time, the series' datapoints slice will be moved back to the pool. -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: transparant aggregations set PNGroups to the pointer value of that function, to uniquely identify any further data requests that will be fed into the same transparant 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. +# Considerations around Series changes and reuse and why we chose copy-on-write. -## Management of point slices +## introduction The `models.Series` type, even when passed by value, has a few fields that need special attention: * `Datapoints []schema.Point` @@ -34,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: @@ -58,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 lirbrary 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. @@ -70,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: @@ -139,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/docs/render-path.md b/docs/render-path.md index 32ac15823e..81ef3da16c 100644 --- a/docs/render-path.md +++ b/docs/render-path.md @@ -92,3 +92,23 @@ TODO talk about planRequests -> getTargets -> mergeSeries -> sort Series -> plan.Run (executes functions and does MDP consolidation with nudging) talk more about what happens at each step, how data is manipulated etc + +## MDP-optimization + +MDP at the leaf of the expr tree (fetch request) 0 means don't optimize, set it to >0 means, 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 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 to 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. + From 1c17a8a4094ab8135ddfd8c56917efb2dfd0158e Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sun, 19 Jan 2020 10:28:31 +0100 Subject: [PATCH 12/46] random clarification related to COW --- api/models/series.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/models/series.go b/api/models/series.go index 879a3d25e7..d99b0cb708 100644 --- a/api/models/series.go +++ b/api/models/series.go @@ -79,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 From 8f11677ffe0491120ff616b8b27f80aa17764028 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sun, 19 Jan 2020 18:57:18 +0100 Subject: [PATCH 13/46] clarify --- expr/parse.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 { From 9b2db02a3b624efc0050c634f7137f646a98253c Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sun, 19 Jan 2020 22:38:30 +0100 Subject: [PATCH 14/46] clarify limitations of pre-canonical, canonicalize when runtime normalizing --- api/dataprocessor.go | 6 +++--- docs/render-path.md | 4 ++++ expr/normalize.go | 39 +++++++++++++++++++++++++++++++++++++-- 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 8544da171f..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) { @@ -407,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) { @@ -773,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/docs/render-path.md b/docs/render-path.md index 81ef3da16c..df605d9c4f 100644 --- a/docs/render-path.md +++ b/docs/render-path.md @@ -56,6 +56,10 @@ 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. + ## nudging diff --git a/expr/normalize.go b/expr/normalize.go index 258d1d88b3..2d5565a24d 100644 --- a/expr/normalize.go +++ b/expr/normalize.go @@ -1,6 +1,8 @@ package expr import ( + "fmt" + "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/consolidation" "github.com/grafana/metrictank/schema" @@ -43,14 +45,47 @@ func normalizeTwo(cache map[Req][]models.Series, a, b models.Series) (models.Ser } // normalizeTo normalizes the given series to the desired interval +// also, any points that would lead to a non-canonical result are removed. // 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 - // TODO verify that Consolidate()'s behavior around odd-sized inputs makes sense. reread old stuff re canonical form etc datapoints := pointSlicePool.Get().([]schema.Point) - datapoints = append(datapoints, in.Datapoints...) + + // scroll to the first point for which ts-in.Interval % interval = 0 + // (e.g. if in.Interval 10, interval 30, and series [60,70,80,90,...] we want to start at 70, such that (70,80,90) forms the first point with ts=90 + + i := 0 + var found bool + for i <= len(in.Datapoints) { + if in.Datapoints[i].Ts-in.Interval%interval == 0 { + found = true + break + } + i++ + } + if !found { + panic(fmt.Sprintf("series %q cannot be normalized from interval %d to %d because it is too short. please request a longer timeframe", in.Target, in.Interval, interval)) + } + + // scroll back to the last point for which ts-in.Interval % interval = 0, which would be the first point to exclude. + // (e.g. if in.Interval 10, interval 30, and series [...,110,120,130,140] we want to exclude 130 and end at 120, such that (100,110,120) forms the last point with ts=120. + + j := len(in.Datapoints) - 1 + found = false + for j >= 0 { + if in.Datapoints[j].Ts-in.Interval%interval == 0 { + found = true + break + } + j-- + } + if !found || j == i { + panic(fmt.Sprintf("series %q cannot be normalized from interval %d to %d because it is too short. please request a longer timeframe", in.Target, in.Interval, interval)) + } + + datapoints = append(datapoints, in.Datapoints[i:j]...) in.Datapoints = consolidation.Consolidate(datapoints, interval/in.Interval, in.Consolidator) in.Interval = interval / in.Interval cache[Req{}] = append(cache[Req{}], in) From 35a725c22717f1088b1f6e259a15b28c1d13513d Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sun, 19 Jan 2020 22:39:55 +0100 Subject: [PATCH 15/46] WIP fix tests --- expr/plan_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/expr/plan_test.go b/expr/plan_test.go index cbe5119347..79b5b216a7 100644 --- a/expr/plan_test.go +++ b/expr/plan_test.go @@ -545,7 +545,7 @@ 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, false) if err != nil { t.Fatal(err) } @@ -715,7 +715,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, false) if err != nil { t.Fatal(err) } @@ -820,7 +820,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, false) if err != c.expectedPlanError { t.Fatalf("case %q: expected plan error %q but got %q", c.testDescription, c.expectedPlanError, err) } From cb220e8a78dee394e91ffd916106b2eb8a1be840 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 18:33:43 +0100 Subject: [PATCH 16/46] deal with non-pre-canonical normalizing like graphite does include more points that may possibly be less accurate it's simpler too. --- expr/normalize.go | 57 ++++++++++++++++++++++------------------------- 1 file changed, 27 insertions(+), 30 deletions(-) diff --git a/expr/normalize.go b/expr/normalize.go index 2d5565a24d..0ff34db93c 100644 --- a/expr/normalize.go +++ b/expr/normalize.go @@ -2,6 +2,7 @@ package expr import ( "fmt" + "math" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/consolidation" @@ -45,49 +46,45 @@ func normalizeTwo(cache map[Req][]models.Series, a, b models.Series) (models.Ser } // normalizeTo normalizes the given series to the desired interval -// also, any points that would lead to a non-canonical result are removed. // 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) - // scroll to the first point for which ts-in.Interval % interval = 0 - // (e.g. if in.Interval 10, interval 30, and series [60,70,80,90,...] we want to start at 70, such that (70,80,90) forms the first point with ts=90 - - i := 0 - var found bool - for i <= len(in.Datapoints) { - if in.Datapoints[i].Ts-in.Interval%interval == 0 { - found = true - break - } - i++ - } - if !found { - panic(fmt.Sprintf("series %q cannot be normalized from interval %d to %d because it is too short. please request a longer timeframe", in.Target, in.Interval, interval)) + 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)) } - // scroll back to the last point for which ts-in.Interval % interval = 0, which would be the first point to exclude. - // (e.g. if in.Interval 10, interval 30, and series [...,110,120,130,140] we want to exclude 130 and end at 120, such that (100,110,120) forms the last point with ts=120. + // 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! - j := len(in.Datapoints) - 1 - found = false - for j >= 0 { - if in.Datapoints[j].Ts-in.Interval%interval == 0 { - found = true - break - } - j-- - } - if !found || j == i { - panic(fmt.Sprintf("series %q cannot be normalized from interval %d to %d because it is too short. please request a longer timeframe", in.Target, in.Interval, interval)) + 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[i:j]...) + datapoints = append(datapoints, in.Datapoints...) + in.Datapoints = consolidation.Consolidate(datapoints, interval/in.Interval, in.Consolidator) - in.Interval = interval / in.Interval + 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 +} From 4a7de6a154e9a605ac826d51f2e760296368e1b1 Mon Sep 17 00:00:00 2001 From: Robert Milan Date: Mon, 20 Jan 2020 19:15:03 +0100 Subject: [PATCH 17/46] add msgp generate directives to request for PNGroup --- api/models/request.go | 3 ++ api/models/request_gen.go | 59 ++++++++++++++++++++++++++++++++++ api/models/request_gen_test.go | 3 ++ 3 files changed, 65 insertions(+) create mode 100644 api/models/request_gen.go create mode 100644 api/models/request_gen_test.go diff --git a/api/models/request.go b/api/models/request.go index 0f8c895188..16c7ee8f07 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -14,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: 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. From 6e7a145e71919315f82850ece25ef0c0bc6d7a74 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 21:17:56 +0100 Subject: [PATCH 18/46] make PN- and MDP-optimizations separately configurable --- api/config.go | 6 ++++-- api/graphite.go | 4 ++-- cmd/mt-explain/main.go | 6 ++++-- docker/docker-chaos/metrictank.ini | 6 ++++-- docker/docker-cluster-query/metrictank.ini | 6 ++++-- docker/docker-cluster/metrictank.ini | 6 ++++-- .../metrictank.ini | 6 ++++-- docs/config.md | 6 ++++-- expr/funcs.go | 12 +++++------ expr/plan.go | 21 ++++++++++++------- metrictank-sample.ini | 6 ++++-- scripts/config/metrictank-docker.ini | 6 ++++-- scripts/config/metrictank-package.ini | 6 ++++-- 13 files changed, 62 insertions(+), 35 deletions(-) diff --git a/api/config.go b/api/config.go index 5f24689e1c..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,7 +29,7 @@ var ( getTargetsConcurrency int tagdbDefaultLimit uint speculationThreshold float64 - fetchOptimizations bool + optimizations expr.Optimizations graphiteProxy *httputil.ReverseProxy timeZone *time.Location @@ -50,7 +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(&fetchOptimizations, "fetch-optimizations", false, "enable MaxDataPoints optimization and pre-normalization optimizations") + 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/graphite.go b/api/graphite.go index 2a7eb76231..862e1e85ef 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -225,7 +225,7 @@ 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, fetchOptimizations) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, optimizations) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { if request.NoProxy { @@ -1420,7 +1420,7 @@ 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, fetchOptimizations) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, optimizations) if err != nil { response.Write(ctx, response.NewError(http.StatusBadRequest, err.Error())) return diff --git a/cmd/mt-explain/main.go b/cmd/mt-explain/main.go index 8c5fd099a0..c8f8fc96f7 100644 --- a/cmd/mt-explain/main.go +++ b/cmd/mt-explain/main.go @@ -25,7 +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)") - fetchOptimizations := flag.Bool("fetch-optimization", false, "enable MaxDataPoints optimization and pre-normalization optimizations") + 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") @@ -78,7 +80,7 @@ func main() { return } - plan, err := expr.NewPlan(exps, fromUnix, toUnix, uint32(*mdp), *stable, *fetchOptimizations) + 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/docker/docker-chaos/metrictank.ini b/docker/docker-chaos/metrictank.ini index 88a81e7329..959629e395 100644 --- a/docker/docker-chaos/metrictank.ini +++ b/docker/docker-chaos/metrictank.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 a1ac41efe6..929c62a2d5 100644 --- a/docker/docker-cluster-query/metrictank.ini +++ b/docker/docker-cluster-query/metrictank.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 350ae7038b..eb6bfdb640 100644 --- a/docker/docker-cluster/metrictank.ini +++ b/docker/docker-cluster/metrictank.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 7c29680fe4..1d5c0cac54 100644 --- a/docker/docker-dev-custom-cfg-kafka/metrictank.ini +++ b/docker/docker-dev-custom-cfg-kafka/metrictank.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 402506d19e..5e17b14114 100644 --- a/docs/config.md +++ b/docs/config.md @@ -293,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ``` ## metric data inputs ## diff --git a/expr/funcs.go b/expr/funcs.go index ca95f0d23d..82e66dfc5f 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -7,12 +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 - 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 - fetchOptimizations bool // disable PNGroups and MDP + 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/plan.go b/expr/plan.go index 24f0f15250..ed79006be5 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -9,6 +9,11 @@ import ( "github.com/grafana/metrictank/errors" ) +type Optimizations struct { + PreNormalization bool + MDP bool +} + // 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 @@ -37,8 +42,10 @@ func NewReqFromContext(query string, c Context) Req { To: c.to, Cons: c.consol, } - if c.fetchOptimizations { + if c.optimizations.PreNormalization { r.PNGroup = c.PNGroup + } + if c.optimizations.MDP { r.MDP = c.MDP } return r @@ -117,7 +124,7 @@ 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, fetchOptimizations bool) (Plan, error) { +func NewPlan(exprs []*expr, from, to, mdp uint32, stable bool, optimizations Optimizations) (Plan, error) { plan := Plan{ exprs: exprs, MaxDataPoints: mdp, @@ -126,11 +133,11 @@ func NewPlan(exprs []*expr, from, to, mdp uint32, stable, fetchOptimizations boo } for _, e := range exprs { context := Context{ - from: from, - to: to, - MDP: mdp, - PNGroup: 0, // making this explicit here for easy code grepping - fetchOptimizations: fetchOptimizations, + 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, plan.Reqs) if err != nil { diff --git a/metrictank-sample.ini b/metrictank-sample.ini index ca7bb221b0..fd0cdf65aa 100644 --- a/metrictank-sample.ini +++ b/metrictank-sample.ini @@ -248,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 08ed99ed5b..aee15c6f25 100644 --- a/scripts/config/metrictank-docker.ini +++ b/scripts/config/metrictank-docker.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# 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 1b0c04b970..baefe0527f 100644 --- a/scripts/config/metrictank-package.ini +++ b/scripts/config/metrictank-package.ini @@ -244,8 +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 MaxDataPoints optimization and pre-normalization optimizations -fetch-optimizations = false +# enable pre-normalization optimization +pre-normalization = true +# enable MaxDataPoints optimization (experimental) +mdp-optimization = false ## metric data inputs ## From 80001d424fab7dc5e733a33bcdf99dcdee7dff12 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 22:39:20 +0100 Subject: [PATCH 19/46] make it simpler to use models.Series in tests --- expr/data_test.go | 17 +++++ expr/func_aggregate_test.go | 145 ++++++++---------------------------- 2 files changed, 46 insertions(+), 116 deletions(-) diff --git a/expr/data_test.go b/expr/data_test.go index b2171b9073..ba14f1dc91 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,19 @@ 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, + } +} 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, ) } From 9e7156cb7e24957f93a95126e020fd685e7c29b7 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 22:53:10 +0100 Subject: [PATCH 20/46] fix dataprocessor tests --- api/dataprocessor_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/api/dataprocessor_test.go b/api/dataprocessor_test.go index 8d09895280..9dfaadd3ff 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,12 +598,12 @@ 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 := models.NewReq(key, "", "", from, to, maxPoints, rawInterval, 0, consolidator, 0, cluster.Manager.ThisNode(), schemaId, aggId) req.Archive = archive req.ArchInterval = archInterval req.TTL = ttl From bc739173c0540f58de98da08e8e5ab9b5cd0ffcd Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 22:53:37 +0100 Subject: [PATCH 21/46] Newreq for dataprocessor and --- api/models/request.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/api/models/request.go b/api/models/request.go index 16c7ee8f07..1ffc57bf9f 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -52,6 +52,25 @@ type Req struct { // 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 minus 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, + Pattern: patt, + From: from, + To: to, + MaxPoints: maxPoints, + PNGroup: pngroup, + RawInterval: rawInterval, + Consolidator: cons, + ConsReq: consReq, + Node: node, + SchemaId: schemaId, + AggId: aggId, + } +} + // Init initializes a request based on the metadata that we know of. // It sets all properties minus the ones that need request planning func (r *Req) Init(archive idx.Archive, cons consolidation.Consolidator, node cluster.Node) { From 29a405499062ff6336eaae626e087ee1fc5850b5 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 22:54:44 +0100 Subject: [PATCH 22/46] testPlan tests --- api/query_engine_test.go | 108 +++++++++++++++++++++------------------ 1 file changed, 58 insertions(+), 50 deletions(-) diff --git a/api/query_engine_test.go b/api/query_engine_test.go index 729b688306..4af18c6483 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -12,8 +12,16 @@ 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) +func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { var schemas []conf.Schema oriMaxPointsPerReqSoft := maxPointsPerReqSoft oriMaxPointsPerHardReq := maxPointsPerReqHard @@ -32,16 +40,17 @@ func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models maxPointsPerReqHard = maxPointsPerReqSoft * 10 mdata.Schemas = conf.NewSchemas(schemas) - out, _, _, err := alignRequests(now, reqs[0].From, reqs[0].To, reqs) + out, err := planRequests(now, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) 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)) + if int(out.cnt) != len(outReqs) { + t.Errorf("different number of requests expected: %v, got: %v", len(outReqs), out.cnt) } else { + got := out.List() 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 !exp.Equals(got[r]) { + t.Errorf("request %d:\nexpected: %v\n got: %v", r, exp.DebugString(), got[r].DebugString()) } } } @@ -51,8 +60,8 @@ func testAlign(reqs []models.Req, retentions []conf.Retentions, outReqs []models } // 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{ +func TestPlanRequestsBasic(t *testing.T) { + testPlan([]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), }, @@ -72,8 +81,8 @@ func TestAlignRequestsBasic(t *testing.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{ +func TestPlanRequestsBasicDiff(t *testing.T) { + testPlan([]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), }, @@ -97,8 +106,8 @@ func TestAlignRequestsBasicDiff(t *testing.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{ +func TestPlanRequestsAlerting(t *testing.T) { + testPlan([]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), }, @@ -121,8 +130,8 @@ func TestAlignRequestsAlerting(t *testing.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{ +func TestPlanRequestsBasicBestEffort(t *testing.T) { + testPlan([]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), }, @@ -145,8 +154,8 @@ func TestAlignRequestsBasicBestEffort(t *testing.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{ +func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { + testPlan([]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), }, @@ -167,8 +176,8 @@ func TestAlignRequestsMultipleIntervalsPerSchema(t *testing.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{ +func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { + testPlan([]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), }, @@ -189,8 +198,8 @@ func TestAlignRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.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{ +func TestPlanRequestsHalfGood(t *testing.T) { + testPlan([]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), }, @@ -214,8 +223,8 @@ func TestAlignRequestsHalfGood(t *testing.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{ +func TestPlanRequestsGoodRollup(t *testing.T) { + testPlan([]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), }, @@ -240,8 +249,8 @@ func TestAlignRequestsGoodRollup(t *testing.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{ +func TestPlanRequestsDiffGoodRollup(t *testing.T) { + testPlan([]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), }, @@ -266,8 +275,8 @@ func TestAlignRequestsDiffGoodRollup(t *testing.T) { } // 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{ +func TestPlanRequestsWeird(t *testing.T) { + testPlan([]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), }, @@ -291,8 +300,8 @@ func TestAlignRequestsWeird(t *testing.T) { } // now TTL of first rollup is *just* enough -func TestAlignRequestsWeird2(t *testing.T) { - testAlign([]models.Req{ +func TestPlanRequestsWeird2(t *testing.T) { + testPlan([]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), }, @@ -317,8 +326,8 @@ func TestAlignRequestsWeird2(t *testing.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{ +func TestPlanRequestsNoOtherChoice(t *testing.T) { + testPlan([]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), }, @@ -343,8 +352,8 @@ func TestAlignRequestsNoOtherChoice(t *testing.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{ +func TestPlanRequests3rdBand(t *testing.T) { + testPlan([]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), }, @@ -370,8 +379,8 @@ func TestAlignRequests3rdBand(t *testing.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{ +func TestPlanRequests2RollupsDisabled(t *testing.T) { + testPlan([]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), }, @@ -395,8 +404,8 @@ func TestAlignRequests2RollupsDisabled(t *testing.T) { t, ) } -func TestAlignRequestsHuh(t *testing.T) { - testAlign([]models.Req{ +func TestPlanRequestsHuh(t *testing.T) { + testPlan([]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), }, @@ -421,8 +430,8 @@ func TestAlignRequestsHuh(t *testing.T) { ) } -func TestAlignRequestsDifferentReadyStates(t *testing.T) { - testAlign([]models.Req{ +func TestPlanRequestsDifferentReadyStates(t *testing.T) { + testPlan([]models.Req{ reqRaw(test.GetMKey(1), 100, 300, 800, 1, consolidation.Avg, 0, 0), }, []conf.Retentions{ @@ -461,10 +470,10 @@ func testMaxPointsPerReq(maxPointsSoft, maxPointsHard int, reqs []models.Req, t ), }}) - out, _, _, err := alignRequests(30*day, reqs[0].From, reqs[0].To, reqs) + out, err := planRequests(30*day, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) maxPointsPerReqSoft = origMaxPointsPerReqSoft maxPointsPerReqHard = origMaxPointsPerReqHard - return out, err + return out.List(), err } func TestGettingOneNextBiggerAgg(t *testing.T) { @@ -511,15 +520,14 @@ func TestMaxPointsPerReqHardLimit(t *testing.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 BenchmarkPlanRequestsSamePNGroup(b *testing.B) { + var res *ReqsPlan + reqs := NewReqMap() + reqs.Add(reqRaw(test.GetMKey(1), 0, 3600*24*7, 1000, 10, consolidation.Avg, 0, 0)) + reqs.Add(reqRaw(test.GetMKey(2), 0, 3600*24*7, 1000, 30, consolidation.Avg, 4, 0)) + reqs.Add(reqRaw(test.GetMKey(3), 0, 3600*24*7, 1000, 60, consolidation.Avg, 8, 0)) mdata.Schemas = conf.NewSchemas([]conf.Schema{ { Pattern: regexp.MustCompile("a"), @@ -551,7 +559,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) } result = res } From c863a427e07896d7320901f26a63e9df867cee27 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 22:54:59 +0100 Subject: [PATCH 23/46] update consolidaton tests for NewPlan taking optimizations --- expr/plan_test.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/expr/plan_test.go b/expr/plan_test.go index 79b5b216a7..3c5950a58a 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" ) @@ -440,6 +441,9 @@ func TestArgInIntKeyword(t *testing.T) { // 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) +// with PN-optimization enabled/disabled. +// (enabling MDP leads to MDP being set on the reqs which means we would have to do all test twice basically +// and MDP is experimental, so we don't check for it here) func TestConsolidateBy(t *testing.T) { from := uint32(1000) to := uint32(2000) @@ -545,15 +549,15 @@ 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, false) + 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): {{ @@ -715,7 +719,7 @@ func TestNamingChains(t *testing.T) { if err != nil { t.Fatal(err) } - plan, err := NewPlan(exprs, from, to, 800, stable, false) + plan, err := NewPlan(exprs, from, to, 800, stable, Optimizations{}) if err != nil { t.Fatal(err) } @@ -726,6 +730,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 +825,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, false) + _, 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) } From 8ba064f2acae876d8231aa87aac9faba0afd6510 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 23:37:28 +0100 Subject: [PATCH 24/46] getModel(): set interval! --- expr/data_test.go | 11 +++++++++++ expr/func_groupbytags_test.go | 10 ---------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/expr/data_test.go b/expr/data_test.go index ba14f1dc91..769c751829 100644 --- a/expr/data_test.go +++ b/expr/data_test.go @@ -264,3 +264,14 @@ func getTargetSeries(target string, data []schema.Point) models.Series { 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_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{ From c041be3773c6597121e573e11eafbc6597cf9dee Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 20 Jan 2020 23:39:38 +0100 Subject: [PATCH 25/46] fix TestConsolidateBy --- expr/plan_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/expr/plan_test.go b/expr/plan_test.go index 3c5950a58a..67858c400a 100644 --- a/expr/plan_test.go +++ b/expr/plan_test.go @@ -564,21 +564,25 @@ func TestConsolidateBy(t *testing.T) { 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): {{ QueryPatt: "a", Target: "a", Consolidator: consolidation.Min, + Interval: 10, }}, NewReq("a", from, to, consolidation.Sum): {{ QueryPatt: "a", Target: "a", Consolidator: consolidation.Sum, + Interval: 10, }}, NewReq("b", from, to, consolidation.Max): {{ QueryPatt: "b", Target: "b", Consolidator: consolidation.Max, + Interval: 10, }}, } out, err := plan.Run(input) From 95d51c8a445aaef3b407eaf1b31630deacad55be Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Tue, 21 Jan 2020 12:57:22 +0100 Subject: [PATCH 26/46] fix tests: MDP is now used to mark MDP optimizations, don't set it --- api/query_engine_test.go | 136 +++++++++++++++++++-------------------- 1 file changed, 68 insertions(+), 68 deletions(-) diff --git a/api/query_engine_test.go b/api/query_engine_test.go index 4af18c6483..cf8a0a3f09 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -62,8 +62,8 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. // 2 series requested with equal raw intervals. req 0-30. now 1200. one archive of ttl=1200 does it func TestPlanRequestsBasic(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 0, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -71,8 +71,8 @@ func TestPlanRequestsBasic(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), }, nil, 1200, @@ -83,8 +83,8 @@ func TestPlanRequestsBasic(t *testing.T) { // 2 series requested with equal raw intervals from different schemas. req 0-30. now 1200. their archives of ttl=1200 do it func TestPlanRequestsBasicDiff(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -95,8 +95,8 @@ func TestPlanRequestsBasicDiff(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), }, nil, 1200, @@ -108,8 +108,8 @@ func TestPlanRequestsBasicDiff(t *testing.T) { // (real example seen with alerting queries) func TestPlanRequestsAlerting(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -120,8 +120,8 @@ func TestPlanRequestsAlerting(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1200, 60, 6), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), }, nil, 1200, @@ -132,8 +132,8 @@ func TestPlanRequestsAlerting(t *testing.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 TestPlanRequestsBasicBestEffort(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -144,8 +144,8 @@ func TestPlanRequestsBasicBestEffort(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 60, 6), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), }, nil, 1200, @@ -156,8 +156,8 @@ func TestPlanRequestsBasicBestEffort(t *testing.T) { // 2 series requested with different raw intervals from the same schemas. Both requests should use the 60second rollups func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -166,8 +166,8 @@ func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 60, 800, 60, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), }, nil, 1200, @@ -178,8 +178,8 @@ func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { // 2 series requested with different raw intervals from the same schemas. Both requests should use raw func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -188,8 +188,8 @@ func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 30, 3), + reqOut(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0, 0, 30, 800, 30, 1), }, nil, 800, @@ -200,8 +200,8 @@ func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.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 TestPlanRequestsHalfGood(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -213,8 +213,8 @@ func TestPlanRequestsHalfGood(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 120, 12), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 1, 120, 1200, 120, 1), }, nil, 1200, @@ -225,8 +225,8 @@ func TestPlanRequestsHalfGood(t *testing.T) { // 2 series requested with different raw intervals from different schemas. req 0-30. now 1200. both have short raw + good rollup func TestPlanRequestsGoodRollup(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -239,8 +239,8 @@ func TestPlanRequestsGoodRollup(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), }, nil, 1200, @@ -251,8 +251,8 @@ func TestPlanRequestsGoodRollup(t *testing.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 TestPlanRequestsDiffGoodRollup(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -265,8 +265,8 @@ func TestPlanRequestsDiffGoodRollup(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 100, 1200, 600, 6), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 600, 1200, 600, 1), }, nil, 1200, @@ -277,8 +277,8 @@ func TestPlanRequestsDiffGoodRollup(t *testing.T) { // now raw is short and we have a rollup we can use instead, at same interval as one of the raws func TestPlanRequestsWeird(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -290,8 +290,8 @@ func TestPlanRequestsWeird(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 60, 1200, 60, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 0, 60, 1200, 60, 1), }, nil, 1200, @@ -302,8 +302,8 @@ func TestPlanRequestsWeird(t *testing.T) { // now TTL of first rollup is *just* enough func TestPlanRequestsWeird2(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -316,8 +316,8 @@ func TestPlanRequestsWeird2(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), }, nil, 1200, @@ -328,8 +328,8 @@ func TestPlanRequestsWeird2(t *testing.T) { // now TTL of first rollup is not enough but we have no other choice but to use it func TestPlanRequestsNoOtherChoice(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -342,8 +342,8 @@ func TestPlanRequestsNoOtherChoice(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1199, 120, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1199, 120, 1), }, nil, 1200, @@ -354,8 +354,8 @@ func TestPlanRequestsNoOtherChoice(t *testing.T) { // now TTL of first rollup is not enough and we have a 3rd band to use func TestPlanRequests3rdBand(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -369,8 +369,8 @@ func TestPlanRequests3rdBand(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 2, 240, 1200, 240, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 1, 240, 1200, 240, 1), }, nil, 1200, @@ -381,8 +381,8 @@ func TestPlanRequests3rdBand(t *testing.T) { // now TTL of raw/first rollup is not enough but the two rollups are disabled, so must use raw func TestPlanRequests2RollupsDisabled(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -396,8 +396,8 @@ func TestPlanRequests2RollupsDisabled(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1100, 60, 6), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 60, 1), }, nil, 1200, @@ -406,8 +406,8 @@ func TestPlanRequests2RollupsDisabled(t *testing.T) { } func TestPlanRequestsHuh(t *testing.T) { testPlan([]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), + reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -421,8 +421,8 @@ func TestPlanRequestsHuh(t *testing.T) { ), }, []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), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1080, 120, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 120, 2), }, nil, 1200, @@ -432,7 +432,7 @@ func TestPlanRequestsHuh(t *testing.T) { func TestPlanRequestsDifferentReadyStates(t *testing.T) { testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 100, 300, 800, 1, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(1), 100, 300, 0, 1, consolidation.Avg, 0, 0), }, []conf.Retentions{ conf.BuildFromRetentions( @@ -444,7 +444,7 @@ func TestPlanRequestsDifferentReadyStates(t *testing.T) { ), }, []models.Req{ - reqOut(test.GetMKey(1), 100, 300, 800, 1, consolidation.Avg, 0, 0, 4, 60, 480, 60, 1), + reqOut(test.GetMKey(1), 100, 300, 0, 1, consolidation.Avg, 0, 0, 4, 60, 480, 60, 1), }, nil, 500, @@ -478,7 +478,7 @@ func testMaxPointsPerReq(maxPointsSoft, maxPointsHard int, reqs []models.Req, t 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), + reqOut(test.GetMKey(1), 29*day, 30*day, 0, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), } // without maxPointsPerReqSoft = 23*hour we'd get archive 0 for this request, @@ -494,7 +494,7 @@ func TestGettingOneNextBiggerAgg(t *testing.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), + reqOut(test.GetMKey(1), 29*day, 30*day, 0, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), } // maxPointsPerReqSoft only allows 24 points, so the aggregation 2 with @@ -510,7 +510,7 @@ func TestGettingTwoNextBiggerAgg(t *testing.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), + reqOut(test.GetMKey(1), 29*day, 30*day, 0, 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 @@ -525,9 +525,9 @@ var result *ReqsPlan func BenchmarkPlanRequestsSamePNGroup(b *testing.B) { var res *ReqsPlan reqs := NewReqMap() - reqs.Add(reqRaw(test.GetMKey(1), 0, 3600*24*7, 1000, 10, consolidation.Avg, 0, 0)) - reqs.Add(reqRaw(test.GetMKey(2), 0, 3600*24*7, 1000, 30, consolidation.Avg, 4, 0)) - reqs.Add(reqRaw(test.GetMKey(3), 0, 3600*24*7, 1000, 60, consolidation.Avg, 8, 0)) + 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"), From 83da44d3b6c3f0e8615391be69b692435f7c3a3b Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Tue, 21 Jan 2020 15:53:55 +0100 Subject: [PATCH 27/46] fix planRequests tests to match new behavior --- api/query_engine_test.go | 49 ++++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/api/query_engine_test.go b/api/query_engine_test.go index cf8a0a3f09..6aec306968 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -59,7 +59,7 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. maxPointsPerReqHard = oriMaxPointsPerHardReq } -// 2 series requested with equal raw intervals. req 0-30. now 1200. one archive of ttl=1200 does it +// 2 series with equal schema of 1 raw archive. tsRange within TTL of raw.return the raw data func TestPlanRequestsBasic(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), @@ -80,7 +80,7 @@ func TestPlanRequestsBasic(t *testing.T) { ) } -// 2 series requested with equal raw intervals from different schemas. req 0-30. now 1200. their archives of ttl=1200 do it +// 2 series with distinct but equal schemas of 1 raw archive. tsRange within TTL for both. return the raw data. func TestPlanRequestsBasicDiff(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), @@ -104,9 +104,8 @@ func TestPlanRequestsBasicDiff(t *testing.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 TestPlanRequestsAlerting(t *testing.T) { +// 2 series with distinct schemas of different raw archive. tsRange within TTL for both. return them at their native intervals +func TestPlanRequestsDifferentIntervals(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), @@ -120,7 +119,7 @@ func TestPlanRequestsAlerting(t *testing.T) { ), }, []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1200, 60, 6), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1200, 10, 1), reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), }, nil, @@ -129,7 +128,7 @@ func TestPlanRequestsAlerting(t *testing.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 +// 2 series with distinct schemas of 1 raw archive with different interval and TTL . tsRange within TTL for only one of them. return them at their native intervals, because there is no rollup func TestPlanRequestsBasicBestEffort(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), @@ -144,7 +143,7 @@ func TestPlanRequestsBasicBestEffort(t *testing.T) { ), }, []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 60, 6), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), }, nil, @@ -153,46 +152,46 @@ func TestPlanRequestsBasicBestEffort(t *testing.T) { ) } -// 2 series requested with different raw intervals from the same schemas. Both requests should use the 60second rollups -func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { +// 2 series with different raw intervals from the same schemas. Both requests should use raw +func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0), }, []conf.Retentions{ conf.BuildFromRetentions( - conf.NewRetentionMT(1, 800, 0, 0, 0), - conf.NewRetentionMT(60, 1100, 0, 0, 0), + conf.NewRetentionMT(10, 800, 0, 0, 0), + conf.NewRetentionMT(60, 1200, 0, 0, 0), ), }, []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 60, 800, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0, 0, 30, 800, 30, 1), }, nil, - 1200, + 800, t, ) } -// 2 series requested with different raw intervals from the same schemas. Both requests should use raw -func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { +// 2 series with different raw intervals from the same schemas. TTL causes both to go to first rollup +func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0), + reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0), }, []conf.Retentions{ conf.BuildFromRetentions( - conf.NewRetentionMT(10, 800, 0, 0, 0), - conf.NewRetentionMT(60, 1200, 0, 0, 0), + conf.NewRetentionMT(1, 800, 0, 0, 0), + conf.NewRetentionMT(60, 1100, 0, 0, 0), ), }, []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 30, 3), - reqOut(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0, 0, 30, 800, 30, 1), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 1, 60, 800, 60, 1), + reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), }, nil, - 800, + 1200, t, ) } @@ -213,7 +212,7 @@ func TestPlanRequestsHalfGood(t *testing.T) { ), }, []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 120, 12), + reqOut(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 1, 120, 1200, 120, 1), }, nil, From 1929cb68a73a387fc92751f1f803a3a44e9f39e3 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Tue, 21 Jan 2020 17:15:30 +0100 Subject: [PATCH 28/46] better query engine tests --- api/query_engine_helpers_test.go | 46 ++++++++ api/query_engine_test.go | 173 ++++++++++++++++--------------- 2 files changed, 134 insertions(+), 85 deletions(-) create mode 100644 api/query_engine_helpers_test.go diff --git a/api/query_engine_helpers_test.go b/api/query_engine_helpers_test.go new file mode 100644 index 0000000000..45043653bd --- /dev/null +++ b/api/query_engine_helpers_test.go @@ -0,0 +1,46 @@ +package api + +import ( + "github.com/grafana/metrictank/api/models" + "github.com/grafana/metrictank/consolidation" + "github.com/grafana/metrictank/test" +) + +type archive struct { + i uint8 + interval uint32 + ttl uint32 + outInterval uint32 + aggNum uint32 +} + +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 6aec306968..a37f6cec75 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -20,6 +20,16 @@ func getReqMap(reqs []models.Req) *ReqMap { return rm } +// different tests +// * one or both may need to be pushed to rollup to meet TTL +// * regardless of TTL, rollup may or may not be available +// * use different native resolution within schemas (e.g. skip archives) +// * ready or not, based on ts +// within a test: +// * whether they have equivalent rules or use exact same rule -> abstract out +// * one or more groups may form due to PNGroup +// * soft limit breach + // testPlan verifies the aligment of the given requests, given the retentions (one or more patterns, one or more retentions each) func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { var schemas []conf.Schema @@ -59,97 +69,89 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. maxPointsPerReqHard = oriMaxPointsPerHardReq } -// 2 series with equal schema of 1 raw archive. tsRange within TTL of raw.return the raw data -func TestPlanRequestsBasic(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 0, 0), - }, - []conf.Retentions{ - conf.BuildFromRetentions( - conf.NewRetentionMT(60, 1200, 0, 0, 0), - ), - }, - []models.Req{ - reqOut(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) +func TestPlanRequests_SameInt_SameTTL_RawOnly_RawMatches(t *testing.T) { + in, out := generate(0, 30, []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, t) + + // also test what happens when two series use distinct, but equal schemas + rets = append(rets, rets[0]) + in[1].AggId, out[1].AggId = 1, 1 + testPlan(in, rets, out, nil, 1200, t) } -// 2 series with distinct but equal schemas of 1 raw archive. tsRange within TTL for both. return the raw data. -func TestPlanRequestsBasicDiff(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 60, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 60, consolidation.Avg, 0, 0, 0, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) +func TestPlanRequests_DifferentInt_SameTTL_RawOnly_RawMatches(t *testing.T) { + in, out := generate(0, 30, []reqProp{ + NewReqProp(10, 0, 0), + NewReqProp(60, 1, 0), + }) + rets := []conf.Retentions{ + conf.MustParseRetentions("10s:1200s:60s:2:true"), + conf.MustParseRetentions("60s:1200s:60s:2:true"), + } + adjust(&out[0], 0, 10, 10, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, t) } -// 2 series with distinct schemas of different raw archive. tsRange within TTL for both. return them at their native intervals -func TestPlanRequestsDifferentIntervals(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1200, 10, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) +//IDEA: similar: but have rollups for both: only 1 serie needs to degrade due to TTL +func TestPlanRequests_DifferentInt_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:1100s:60s:2:true"), + } + adjust(&out[0], 0, 10, 10, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, t) } -// 2 series with distinct schemas of 1 raw archive with different interval and TTL . tsRange within TTL for only one of them. return them at their native intervals, because there is no rollup -func TestPlanRequestsBasicBestEffort(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - t, - ) +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,30s:1000s:60s:2:true"), + conf.MustParseRetentions("60s:1100s:60s:2:true"), + } + adjust(&out[0], 1, 30, 30, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, t) + + // now let's try the same but the archive we need is not ready + + rets2 := []conf.Retentions{ + conf.MustParseRetentions("10s:800s:60s:2:true,30s:1000s:60s:2:false"), + conf.MustParseRetentions("60s:1100s:60s:2:true"), + } + adjust(&out[0], 0, 10, 10, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets2, out, nil, 1200, t) +} + +func TestPlanRequests_DifferentInt_DifferentTTL_1RawOnly1RawAndRollups_1RawMatch1RollupMatch(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:1100s:60s:2:true,5m:2000s:5min:2:true"), // extra rollup that we don't care for + } + adjust(&out[0], 0, 10, 10, 1200) + adjust(&out[1], 0, 60, 60, 1200) + testPlan(in, rets, out, nil, 1200, t) } // 2 series with different raw intervals from the same schemas. Both requests should use raw @@ -175,6 +177,7 @@ func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { } // 2 series with different raw intervals from the same schemas. TTL causes both to go to first rollup +// IDEA: similar but only one has a TTL problem func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { testPlan([]models.Req{ reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), From 44fdd938ccdd8b3f1d31039c025bc477f9012557 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Tue, 21 Jan 2020 20:36:57 +0100 Subject: [PATCH 29/46] fix query engine tests --- api/query_engine_helpers_test.go | 8 - api/query_engine_test.go | 514 ++++++++++--------------------- test/key.go | 4 + 3 files changed, 174 insertions(+), 352 deletions(-) diff --git a/api/query_engine_helpers_test.go b/api/query_engine_helpers_test.go index 45043653bd..2e0de36480 100644 --- a/api/query_engine_helpers_test.go +++ b/api/query_engine_helpers_test.go @@ -6,14 +6,6 @@ import ( "github.com/grafana/metrictank/test" ) -type archive struct { - i uint8 - interval uint32 - ttl uint32 - outInterval uint32 - aggNum uint32 -} - type reqProp struct { raw uint32 schemaID uint16 diff --git a/api/query_engine_test.go b/api/query_engine_test.go index a37f6cec75..cf55549410 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" @@ -20,16 +20,6 @@ func getReqMap(reqs []models.Req) *ReqMap { return rm } -// different tests -// * one or both may need to be pushed to rollup to meet TTL -// * regardless of TTL, rollup may or may not be available -// * use different native resolution within schemas (e.g. skip archives) -// * ready or not, based on ts -// within a test: -// * whether they have equivalent rules or use exact same rule -> abstract out -// * one or more groups may form due to PNGroup -// * soft limit breach - // testPlan verifies the aligment of the given requests, given the retentions (one or more patterns, one or more retentions each) func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { var schemas []conf.Schema @@ -49,18 +39,24 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. } 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) + //spew.Dump(mdata.Schemas) out, err := planRequests(now, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) if err != outErr { t.Errorf("different err value expected: %v, got: %v", outErr, err) } - if int(out.cnt) != len(outReqs) { - t.Errorf("different number of requests expected: %v, got: %v", len(outReqs), out.cnt) - } else { - got := out.List() - 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()) + 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()) + } } } } @@ -69,8 +65,20 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. maxPointsPerReqHard = oriMaxPointsPerHardReq } -func TestPlanRequests_SameInt_SameTTL_RawOnly_RawMatches(t *testing.T) { - in, out := generate(0, 30, []reqProp{ +// 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), }) @@ -83,375 +91,189 @@ func TestPlanRequests_SameInt_SameTTL_RawOnly_RawMatches(t *testing.T) { // also test what happens when two series use distinct, but equal schemas rets = append(rets, rets[0]) - in[1].AggId, out[1].AggId = 1, 1 + in[1].SchemaId, out[1].SchemaId = 1, 1 testPlan(in, rets, out, nil, 1200, 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, 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, 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 } +*/ -func TestPlanRequests_DifferentInt_SameTTL_RawOnly_RawMatches(t *testing.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"), } - adjust(&out[0], 0, 10, 10, 1200) - adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets, out, nil, 1200, t) + t.Run("NoPNGroups", func(t *testing.T) { + testPlan(in, rets, out, 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, 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, t) + }) } -//IDEA: similar: but have rollups for both: only 1 serie needs to degrade due to TTL -func TestPlanRequests_DifferentInt_DifferentTTL_RawOnly_1RawShort(t *testing.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:1100s:60s:2:true"), + conf.MustParseRetentions("60s:1080s:60s:2:true"), } - adjust(&out[0], 0, 10, 10, 1200) - adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets, out, nil, 1200, t) + 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, 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, 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, t) + }) + } -func TestPlanRequests_DifferentInt_DifferentTTL_1RawOnly1RawAndRollups_1RawShort(t *testing.T) { +func TestPlanRequests_DifferentInterval_DifferentTTL_1RawOnly1RawAndRollups_1Raw1Rollup(t *testing.T) { in, out := generate(0, 1000, []reqProp{ NewReqProp(10, 0, 0), - NewReqProp(60, 1, 0), + NewReqProp(60, 2, 0), }) rets := []conf.Retentions{ - conf.MustParseRetentions("10s:800s:60s:2:true,30s:1000s:60s:2:true"), - conf.MustParseRetentions("60s:1100s:60s:2:true"), + 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, 1200) - adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets, out, nil, 1200, t) + 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, t) + }) + + 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, t) + }) - // now let's try the same but the archive we need is not ready + 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, t) + }) - rets2 := []conf.Retentions{ - conf.MustParseRetentions("10s:800s:60s:2:true,30s:1000s:60s:2:false"), - conf.MustParseRetentions("60s:1100s:60s:2:true"), - } - adjust(&out[0], 0, 10, 10, 1200) - adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets2, out, nil, 1200, t) } -func TestPlanRequests_DifferentInt_DifferentTTL_1RawOnly1RawAndRollups_1RawMatch1RollupMatch(t *testing.T) { +// 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:1100s:60s:2:true,5m:2000s:5min:2:true"), // extra rollup that we don't care for + 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, 1200) + adjust(&out[0], 0, 10, 10, 800) adjust(&out[1], 0, 60, 60, 1200) testPlan(in, rets, out, nil, 1200, t) -} - -// 2 series with different raw intervals from the same schemas. Both requests should use raw -func TestPlanRequestsMultiIntervalsWithRuntimeConsolidation(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 30, consolidation.Avg, 0, 0, 0, 30, 800, 30, 1), - }, - nil, - 800, - t, - ) -} - -// 2 series with different raw intervals from the same schemas. TTL causes both to go to first rollup -// IDEA: similar but only one has a TTL problem -func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 60, 800, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 1, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - 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 TestPlanRequestsHalfGood(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 0, 10, 800, 10, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 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 TestPlanRequestsGoodRollup(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), - }, - nil, - 1200, - 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 TestPlanRequestsDiffGoodRollup(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 100, 1200, 600, 6), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 600, 1200, 600, 1), - }, - nil, - 1200, - t, - ) -} - -// now raw is short and we have a rollup we can use instead, at same interval as one of the raws -func TestPlanRequestsWeird(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 60, 1200, 60, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 0, 60, 1200, 60, 1), - }, - nil, - 1200, - t, - ) -} - -// now TTL of first rollup is *just* enough -func TestPlanRequestsWeird2(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1200, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1200, 120, 1), - }, - nil, - 1200, - t, - ) -} - -// now TTL of first rollup is not enough but we have no other choice but to use it -func TestPlanRequestsNoOtherChoice(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1199, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 2, 0, 1, 120, 1199, 120, 1), - }, - nil, - 1200, - t, - ) -} - -// now TTL of first rollup is not enough and we have a 3rd band to use -func TestPlanRequests3rdBand(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 2, 240, 1200, 240, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 1, 240, 1200, 240, 1), - }, - nil, - 1200, - 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, t) + }) } -// now TTL of raw/first rollup is not enough but the two rollups are disabled, so must use raw -func TestPlanRequests2RollupsDisabled(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 0, 10, 1100, 60, 6), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 60, 1), - }, - nil, - 1200, - t, - ) -} -func TestPlanRequestsHuh(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 0, 30, 0, 10, consolidation.Avg, 0, 0), - reqRaw(test.GetMKey(2), 0, 30, 0, 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, 0, 10, consolidation.Avg, 0, 0, 1, 120, 1080, 120, 1), - reqOut(test.GetMKey(2), 0, 30, 0, 60, consolidation.Avg, 3, 0, 0, 60, 1100, 120, 2), - }, - nil, - 1200, - t, - ) +// 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, t) } -func TestPlanRequestsDifferentReadyStates(t *testing.T) { - testPlan([]models.Req{ - reqRaw(test.GetMKey(1), 100, 300, 0, 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, 0, 1, consolidation.Avg, 0, 0, 4, 60, 480, 60, 1), - }, - nil, - 500, - t, - ) +// 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, t) } var hour uint32 = 60 * 60 @@ -475,10 +297,14 @@ func testMaxPointsPerReq(maxPointsSoft, maxPointsHard int, reqs []models.Req, t out, err := planRequests(30*day, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) maxPointsPerReqSoft = origMaxPointsPerReqSoft maxPointsPerReqHard = origMaxPointsPerReqHard + if err != nil { + return []models.Req{}, err + } return out.List(), err } func TestGettingOneNextBiggerAgg(t *testing.T) { + // we ask for 1 day worth, arch & out interval of 1s, ttl of 1h reqs := []models.Req{ reqOut(test.GetMKey(1), 29*day, 30*day, 0, 1, consolidation.Avg, 0, 0, 0, 1, hour, 1, 1), } 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 { From dd85a3da3098e8188c653525d63c294294bb1155 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 13:16:32 +0100 Subject: [PATCH 30/46] Req.Equals should also check TTL. we only used this for unit tests --- api/models/request.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/api/models/request.go b/api/models/request.go index 1ffc57bf9f..cf7ffb9aa0 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -184,12 +184,7 @@ func (r Req) TraceLog(span opentracing.Span) { } // Equals compares all fields of a to b for equality. -// Except -// * TTL (because planRequests 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 { @@ -240,6 +235,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 } From ae2ff092aa1af0eb71919ada62ef4707e65e24d2 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 15:00:11 +0100 Subject: [PATCH 31/46] make maxPointsPerReq{Soft,Hard} explicit args, rather than globals clearer. more robust. --- api/graphite.go | 2 +- api/query_engine.go | 12 ++++---- api/query_engine_test.go | 61 +++++++++++++++++++++------------------- 3 files changed, 39 insertions(+), 36 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index 862e1e85ef..881a1fddd9 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -749,7 +749,7 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) // 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 var rp *ReqsPlan - rp, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints) + rp, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints, maxPointsPerReqSoft, maxPointsPerReqHard) meta.RenderStats.PointsFetch = rp.PointsFetch() meta.RenderStats.PointsReturn = rp.PointsReturn(plan.MaxDataPoints) reqsList := rp.List() diff --git a/api/query_engine.go b/api/query_engine.go index 3089d6919a..8179391899 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -65,7 +65,7 @@ func getRetentions(req models.Req) []conf.Retention { // // note: it is assumed that all requests have the same from & to. // also takes a "now" value which we compare the TTL against -func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan, error) { +func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32, mpprSoft, mpprHard int) (*ReqsPlan, error) { ok, rp := false, NewReqsPlan(*reqs) @@ -97,7 +97,7 @@ func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan } } - if maxPointsPerReqSoft > 0 { + 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 @@ -109,7 +109,7 @@ func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan // 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(maxPointsPerReqSoft) { + 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) @@ -117,7 +117,7 @@ func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan return nil, errUnSatisfiable } rp.pngroups[group] = split - if rp.PointsFetch() <= uint32(maxPointsPerReqSoft) { + if rp.PointsFetch() <= uint32(mpprSoft) { goto HonoredSoft } } @@ -130,7 +130,7 @@ func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan // 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(maxPointsPerReqSoft) { + if rp.PointsFetch() <= uint32(mpprSoft) { goto HonoredSoft } } @@ -139,7 +139,7 @@ func planRequests(now, from, to uint32, reqs *ReqMap, planMDP uint32) (*ReqsPlan } HonoredSoft: - if maxPointsPerReqHard > 0 && int(rp.PointsFetch()) > maxPointsPerReqHard { + if mpprHard > 0 && int(rp.PointsFetch()) > mpprHard { return nil, errMaxPointsPerReq } diff --git a/api/query_engine_test.go b/api/query_engine_test.go index cf55549410..8b0aeb011b 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -21,29 +21,35 @@ func getReqMap(reqs []models.Req) *ReqMap { } // testPlan verifies the aligment of the given requests, given the retentions (one or more patterns, one or more retentions each) -func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models.Req, outErr error, now uint32, t *testing.T) { +// 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 - oriMaxPointsPerHardReq := maxPointsPerReqHard + + 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 = maxPointsPerReqSoft * 10 + 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) //spew.Dump(mdata.Schemas) - out, err := planRequests(now, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) + 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) } @@ -60,9 +66,6 @@ func testPlan(reqs []models.Req, retentions []conf.Retentions, outReqs []models. } } } - - maxPointsPerReqSoft = oriMaxPointsPerReqSoft - maxPointsPerReqHard = oriMaxPointsPerHardReq } // There are a lot of factors to consider. I haven't found a practical way to test all combinations of every factor @@ -87,12 +90,12 @@ func TestPlanRequests_SameInterval_SameTTL_RawOnly_RawMatches(t *testing.T) { } adjust(&out[0], 0, 60, 60, 1200) adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets, out, nil, 1200, t) + 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, t) + 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{ @@ -103,7 +106,7 @@ func TestPlanRequests_SameInterval_SameTTL_RawOnly_RawMatches(t *testing.T) { //spew.Dump(rets) //spew.Dump(in) //spew.Dump(out) - testPlan(in, rets, out, errUnSatisfiable, 1200, t) + 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{ @@ -112,7 +115,7 @@ func TestPlanRequests_SameInterval_SameTTL_RawOnly_RawMatches(t *testing.T) { conf.MustParseRetentions("60s:1200s:60s:2:true"), } { rets[0] = r - testPlan(in, rets, out, nil, 1200, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) } } @@ -138,21 +141,21 @@ func TestPlanRequests_DifferentInterval_SameTTL_RawOnly_RawMatches(t *testing.T) conf.MustParseRetentions("60s:1200s:60s:2:true"), } t.Run("NoPNGroups", func(t *testing.T) { - testPlan(in, rets, out, nil, 1200, t) + testPlan(in, rets, out, nil, 1200, 0, 0, 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, t) + 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, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) }) } @@ -168,21 +171,21 @@ func TestPlanRequests_DifferentInterval_DifferentTTL_RawOnly_1RawShort(t *testin 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, t) + testPlan(in, rets, out, nil, 1200, 0, 0, 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, t) + 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, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) }) } @@ -199,7 +202,7 @@ func TestPlanRequests_DifferentInterval_DifferentTTL_1RawOnly1RawAndRollups_1Raw 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, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) }) t.Run("SameButTTLsNotLongEnough", func(t *testing.T) { @@ -209,7 +212,7 @@ func TestPlanRequests_DifferentInterval_DifferentTTL_1RawOnly1RawAndRollups_1Raw } adjust(&out[0], 1, 30, 30, 1140) adjust(&out[1], 0, 60, 60, 1020) - testPlan(in, rets, out, nil, 1200, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) }) t.Run("ArchiveWeNeedIsNotReady", func(t *testing.T) { @@ -218,7 +221,7 @@ func TestPlanRequests_DifferentInterval_DifferentTTL_1RawOnly1RawAndRollups_1Raw adjust(&out[0], 0, 10, 10, 1080) adjust(&out[1], 0, 60, 60, 1320) //spew.Dump(rets) - testPlan(in, rets, out, nil, 1200, t) + testPlan(in, rets, out, nil, 1200, 0, 0, t) }) } @@ -236,13 +239,13 @@ func TestPlanRequests_DifferentInt_DifferentTTL_1RawOnly1RawAndRollups_1RawShort } adjust(&out[0], 0, 10, 10, 800) adjust(&out[1], 0, 60, 60, 1200) - testPlan(in, rets, out, nil, 1200, t) + 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, t) + testPlan(in, rets, out, nil, 3000, 0, 0, t) }) } @@ -257,7 +260,7 @@ func TestPlanRequestsMultiIntervalsUseRaw(t *testing.T) { } adjust(&out[0], 0, 10, 10, 800) adjust(&out[1], 0, 30, 30, 1200) - testPlan(in, rets, out, nil, 800, t) + testPlan(in, rets, out, nil, 800, 0, 0, t) } // 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 @@ -387,7 +390,7 @@ func BenchmarkPlanRequestsSamePNGroup(b *testing.B) { }) for n := 0; n < b.N; n++ { - res, _ = planRequests(14*24*3600, 0, 3600*24*7, reqs, 0) + res, _ = planRequests(14*24*3600, 0, 3600*24*7, reqs, 0, 0, 0) } result = res } From ca8097861f4376eaecceb57faed87ee347fdd1a1 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 15:02:31 +0100 Subject: [PATCH 32/46] do maxPointsPerReq testing same as other tests easier to understand --- api/dataprocessor_test.go | 9 ---- api/query_engine_test.go | 102 +++++++++++--------------------------- 2 files changed, 28 insertions(+), 83 deletions(-) diff --git a/api/dataprocessor_test.go b/api/dataprocessor_test.go index 9dfaadd3ff..291038ee3d 100644 --- a/api/dataprocessor_test.go +++ b/api/dataprocessor_test.go @@ -602,15 +602,6 @@ func reqRaw(key schema.MKey, from, to, maxPoints, rawInterval uint32, consolidat 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, 0, 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/query_engine_test.go b/api/query_engine_test.go index 8b0aeb011b..5d28a1ea8f 100644 --- a/api/query_engine_test.go +++ b/api/query_engine_test.go @@ -261,6 +261,32 @@ func TestPlanRequestsMultiIntervalsUseRaw(t *testing.T) { 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) + }) + }) } // 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 @@ -276,84 +302,12 @@ func TestPlanRequestsMultipleIntervalsPerSchema(t *testing.T) { 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, t) -} - -var hour uint32 = 60 * 60 -var day uint32 = 24 * hour - -func testMaxPointsPerReq(maxPointsSoft, maxPointsHard int, reqs []models.Req, t *testing.T) ([]models.Req, error) { - origMaxPointsPerReqSoft := maxPointsPerReqSoft - origMaxPointsPerReqHard := maxPointsPerReqHard - maxPointsPerReqSoft = maxPointsSoft - maxPointsPerReqHard = maxPointsHard - - 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), - ), - }}) - - out, err := planRequests(30*day, reqs[0].From, reqs[0].To, getReqMap(reqs), 0) - maxPointsPerReqSoft = origMaxPointsPerReqSoft - maxPointsPerReqHard = origMaxPointsPerReqHard - if err != nil { - return []models.Req{}, err - } - return out.List(), err -} - -func TestGettingOneNextBiggerAgg(t *testing.T) { - // we ask for 1 day worth, arch & out interval of 1s, ttl of 1h - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 0, 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) - } -} - -func TestGettingTwoNextBiggerAgg(t *testing.T) { - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 0, 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) - } -} - -func TestMaxPointsPerReqHardLimit(t *testing.T) { - reqs := []models.Req{ - reqOut(test.GetMKey(1), 29*day, 30*day, 0, 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") - } + testPlan(in, rets, out, nil, 1200, 0, 0, t) } var result *ReqsPlan -func BenchmarkPlanRequestsSamePNGroup(b *testing.B) { +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)) From 163e222291b0e1de68ed8b013a9e739c4affc761 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 19:54:28 +0100 Subject: [PATCH 33/46] tests for normalization --- expr/normalize_test.go | 135 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 135 insertions(+) create mode 100644 expr/normalize_test.go 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) + } +} From 32d7ce29182b6c9b8b5ea2e01153f180c018eb0e Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 20:32:12 +0100 Subject: [PATCH 34/46] expr.NewReq should take PNGroup and MDP fields also this will simplify our testing --- expr/plan.go | 12 +++++++----- expr/plan_test.go | 44 ++++++++++++++++++++++---------------------- 2 files changed, 29 insertions(+), 27 deletions(-) diff --git a/expr/plan.go b/expr/plan.go index ed79006be5..27ed8574ad 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -26,12 +26,14 @@ type Req struct { // 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, + Query: query, + From: from, + To: to, + Cons: cons, + PNGroup: PNGroup, + MDP: MDP, } } diff --git a/expr/plan_test.go b/expr/plan_test.go index 67858c400a..17770bffc9 100644 --- a/expr/plan_test.go +++ b/expr/plan_test.go @@ -33,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, }, @@ -47,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, }, @@ -61,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, }, @@ -76,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, }, @@ -91,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, }, @@ -106,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, }, @@ -148,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, }, @@ -162,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, }, @@ -457,7 +457,7 @@ func TestConsolidateBy(t *testing.T) { { "a", []Req{ - NewReq("a", from, to, 0), + NewReq("a", from, to, 0, 0, 0), }, nil, []models.Series{ @@ -468,7 +468,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{ @@ -479,7 +479,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{ @@ -490,7 +490,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{ @@ -501,7 +501,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{ @@ -512,7 +512,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{ @@ -524,8 +524,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{ @@ -536,8 +536,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{ @@ -560,25 +560,25 @@ func TestConsolidateBy(t *testing.T) { 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, From 4c8a573fca290131a1df5a1c147548a4616f4542 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 22:20:23 +0100 Subject: [PATCH 35/46] unit test for planner optimizations --- expr/plan_test.go | 148 +++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 145 insertions(+), 3 deletions(-) diff --git a/expr/plan_test.go b/expr/plan_test.go index 17770bffc9..f9d85dcf25 100644 --- a/expr/plan_test.go +++ b/expr/plan_test.go @@ -439,11 +439,153 @@ 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) -// with PN-optimization enabled/disabled. -// (enabling MDP leads to MDP being set on the reqs which means we would have to do all test twice basically -// and MDP is experimental, so we don't check for it here) func TestConsolidateBy(t *testing.T) { from := uint32(1000) to := uint32(2000) From 3054f19e24e0607b5cf9921a2c346f31078a3736 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 22:48:56 +0100 Subject: [PATCH 36/46] note --- devdocs/alignrequests-too-course-grained.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/devdocs/alignrequests-too-course-grained.txt b/devdocs/alignrequests-too-course-grained.txt index 4c8f49f35f..1f67a26c5e 100644 --- a/devdocs/alignrequests-too-course-grained.txt +++ b/devdocs/alignrequests-too-course-grained.txt @@ -70,6 +70,9 @@ and then normalizing (consolidating) at runtime - and thus spend less resources 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. +The other benefit of pre-normalization is that 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, such that +when it eventually needs to be normalized at runtime, points at the beginning or end of the series may be less accurate. + ## Proposed changes From 427bb5fe36b8eb34ca26efbfe714190c6d1ca8d8 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 23:18:32 +0100 Subject: [PATCH 37/46] refer to http errors by their name as per contribution docs --- api/query_engine.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/api/query_engine.go b/api/query_engine.go index 8179391899..1480b3feff 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -2,6 +2,7 @@ package api import ( "math" + "net/http" "reflect" "github.com/grafana/metrictank/api/models" @@ -22,8 +23,8 @@ var ( // 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.") ) func getRetentions(req models.Req) []conf.Retention { From 3052eb779e644cf74ae13153e5656a1aac790e3a Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Wed, 22 Jan 2020 23:30:15 +0100 Subject: [PATCH 38/46] mt-explain docs --- docs/tools.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) 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: From 38ba46470700562480982dab5ae5b4b3b00846d7 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Thu, 23 Jan 2020 12:45:54 +0100 Subject: [PATCH 39/46] cleanup docs and devdocs --- devdocs/alignrequests-too-course-grained.txt | 94 -------------- devdocs/render-request-handling.md | 20 +++ docs/render-path.md | 123 +++++++++++++++---- 3 files changed, 120 insertions(+), 117 deletions(-) delete mode 100644 devdocs/alignrequests-too-course-grained.txt diff --git a/devdocs/alignrequests-too-course-grained.txt b/devdocs/alignrequests-too-course-grained.txt deleted file mode 100644 index 1f67a26c5e..0000000000 --- a/devdocs/alignrequests-too-course-grained.txt +++ /dev/null @@ -1,94 +0,0 @@ -## MDP-optimizable - -### Greedy-resolution functions - -A Greedy-resolution function (GR-function) is a certain processing function that requires, or may require, high resolution data 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 MDP-optimizable if we know for sure that the render request does not execute 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. - -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. - -This is a more refined solution of https://github.com/grafana/metrictank/issues/463. -In the past, we MDP-optimized every request, which led to incorrect data when fed into GR-functions. -We corrected it by turning off all MDP-optimizations, which I think led to increased latencies, though we don't have the stats off-hand. -The hope is by re-introducing MDP-optimizations the correct way, we can speed up many requests again. - -## Pre-normalizable - -### 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 simplicty, we don't implement this. - -### Transparent aggregation - -A trans-aggregation is a processing function that aggregates multiple series together in a predictable way (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 - -when data will be used together (e.g. aggregating multiple series together) they will need to have the same resolution. -(note that generally, series do *not* need to have the same resolution. We have been aligning resolutions much too aggressively. see https://github.com/grafana/metrictank/issues/926) -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) -This is the case when series go from fetching to transparent aggregation, possibly with some processing functions - except opaque aggregation(s) or IA-function(s) - in between. - -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. - -The other benefit of pre-normalization is that 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, such that -when it eventually needs to be normalized at runtime, points at the beginning or end of the series may be less accurate. - - -## Proposed changes - -1) Don't align any requests (as in `models.Req`) up front -2) Make sure all aggregation functions can normalize at runtime (if necessary), otherwise they will fail to process multiple input series that now can have different intervals -3) Implement pre-normalization -4) While we're at it, may as well implement MDP-optimization -5) (planning-stage awareness of the output interval of IA-functions, which means we can set up fetching/(pre)normalization in a smarter way) - -Step 1 and 2 will solve our most urgent problem of over-aligning data (https://github.com/grafana/metrictank/issues/926) -However it will probably (?) leave some performance optimizations on the table. Which step 3 and 4 address. It's unclear how urgent step 3 and 4 are, though they aren't too difficult to implement. -Implementing both of them can probably be done in one shot, as solving them is done in a similar way. - -Note that step 1 has a complication: -since we no longer set up all models.Req for a request at once, it's trickier to implement "max-points-per-req-soft" and "max-points-per-req-hard". -Think of it this way: if the number of datapoints fetched in a request is higher than the soft limit, which series should we fetch in a more course way? We can implement a heuristic that picks series that are highest resolution, and fetch the next rollup for them, trying to avoid those series that will be processed by a GR-function - though if we have no other choice, we will do it, as we currently do. We keep making fetches coarser until we fetch the coarsest archives of all series. Then we compare against the hard limit and bail out if the limit is breached. Similar to how we currently do it. -Note that to keep the support for the max-points-per-req-soft/max-points-per-req-hard setting we have to implement tracking of GR-functions which means we can probably just do step 4 while we're at it. - -Step 5 is something that can wait. diff --git a/devdocs/render-request-handling.md b/devdocs/render-request-handling.md index 5c31499f8f..f2270e385d 100644 --- a/devdocs/render-request-handling.md +++ b/devdocs/render-request-handling.md @@ -18,3 +18,23 @@ * `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) 0 means don't optimize, set it to >0 means, 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 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 to 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/docs/render-path.md b/docs/render-path.md index df605d9c4f..9f3621863b 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. @@ -58,7 +58,9 @@ I.O.W. is a series that is fetched in such a way that when it is fed to Consolid 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. +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 @@ -88,31 +90,106 @@ 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 -## request flow +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 +``` -TODO talk about -planRequests -> getTargets -> mergeSeries -> sort Series -> plan.Run (executes functions and does MDP consolidation with nudging) +We explain them in detail below. -talk more about what happens at each step, how data is manipulated etc +## Pre-normalization -## MDP-optimization +First, let's look at some definitions. -MDP at the leaf of the expr tree (fetch request) 0 means don't optimize, set it to >0 means, 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 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 to MDP to 0 on the context (and thus also on any subsequent requests) +### Interval-altering function -## Pre-normalization +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 simplicty, 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 wen we know the normalization won't affect anything else. +This is the case when series go from fetching to transparent aggregation, possibly with some processing functions - except opaque aggregation(s) or IA-function(s) - in between, and +with asPercent in a certain mode (where it has to normalize all inputs), but not with divideSeries where it applies the same divisor to multiple dividend inputs, for example. + +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, such that +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. But this is an edge case +2) uses slightly more of the chunk cache. + +## 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. + +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.) -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. +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. +For this reason, this optimization is **experimental** and disabled by default. From 076a490422e57e021b451b43cd9d79ce3e7b56c5 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Thu, 23 Jan 2020 22:36:36 +0100 Subject: [PATCH 40/46] allow passing optimizations as query parameters --- api/graphite.go | 7 +++++-- api/models/graphite.go | 1 + docs/http-api.md | 1 + expr/plan.go | 22 ++++++++++++++++++++++ 4 files changed, 29 insertions(+), 2 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index 881a1fddd9..a414a4600f 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -225,7 +225,9 @@ 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, optimizations) + + opts := optimizations.ApplyUserPrefs(request.Optimizations) + plan, err := expr.NewPlan(exprs, fromUnix, toUnix, mdp, stable, opts) if err != nil { if fun, ok := err.(expr.ErrUnknownFunction); ok { if request.NoProxy { @@ -1420,7 +1422,8 @@ 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, optimizations) + opts := optimizations.ApplyUserPrefs(request.Optimizations) + 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/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/docs/http-api.md b/docs/http-api.md index 5c8cbbaf82..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)) diff --git a/expr/plan.go b/expr/plan.go index 27ed8574ad..1392ce64bd 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -3,6 +3,7 @@ package expr import ( "fmt" "io" + "strings" "github.com/grafana/metrictank/api/models" "github.com/grafana/metrictank/consolidation" @@ -14,6 +15,27 @@ type Optimizations struct { MDP bool } +func (o Optimizations) ApplyUserPrefs(s string) Optimizations { + if s == "" { + return o + } + o.PreNormalization = false + o.MDP = false + if s == "none" { + return o + } + prefs := strings.Split(s, ",") + for _, pref := range prefs { + if pref == "pn" { + o.PreNormalization = true + } + if pref == "mdp" { + o.MDP = true + } + } + return o +} + // 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 526b44317752c2461b3c3b13d4e3aad7d80b2321 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 24 Jan 2020 18:27:04 +0100 Subject: [PATCH 41/46] asPercent: accommodate an extra case --- expr/func_aspercent.go | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/expr/func_aspercent.go b/expr/func_aspercent.go index 5bc3f28ec6..96bbde64cd 100644 --- a/expr/func_aspercent.go +++ b/expr/func_aspercent.go @@ -43,16 +43,20 @@ func (s *FuncAsPercent) Context(context Context) Context { // 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 + // 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-non-nil, total nil -> divides groupwise - // 5) nodes non-nil, total serieslist -> divides groupwise + // 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 - // note: we can't tell the difference between 1/2 up front, so we play it safe and don't align up front - // the only scenario where PNGroup is safe, is case 3 if s.totalSeries == nil && s.nodes == nil { - context.PNGroup = models.PNGroup(uintptr(unsafe.Pointer(s))) + // 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 From 635a4bbad7d49d97f7159092bd9f7edbdfad17dc Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 24 Jan 2020 18:31:47 +0100 Subject: [PATCH 42/46] sean feedback --- api/graphite_req.go | 2 +- api/models/request.go | 2 +- devdocs/expr.md | 6 +++--- docs/render-path.md | 20 +++++++++++++------- expr/plan.go | 2 ++ util/combinations.go | 16 ++++++++-------- 6 files changed, 28 insertions(+), 20 deletions(-) diff --git a/api/graphite_req.go b/api/graphite_req.go index 02f2fd758c..282cabb163 100644 --- a/api/graphite_req.go +++ b/api/graphite_req.go @@ -8,7 +8,7 @@ import ( // ReqMap is a map of requests of data, // it has single requests for which no pre-normalization effort will be performed, and -// requests are that can be pre-normalized together to the same resolution, bundled by their PNGroup +// 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 diff --git a/api/models/request.go b/api/models/request.go index cf7ffb9aa0..ed844bbc7e 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -126,7 +126,7 @@ func (r *Req) AdjustTo(interval, from uint32, rets []conf.Retention) { // 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 - // divisible by the output interval) but let's not worry about that edge case. + // that is a factor of the output interval) but let's not worry about that edge case. r.PlanNormalization(interval) } diff --git a/devdocs/expr.md b/devdocs/expr.md index 093320e032..61b36ab3fb 100644 --- a/devdocs/expr.md +++ b/devdocs/expr.md @@ -12,7 +12,7 @@ Such functions require special options. see https://github.com/grafana/metrictank/issues/926#issuecomment-559596384 -## implement our copy-o-write approach when dealing with modifying series +## 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. @@ -27,7 +27,7 @@ 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 that during plan cleanup time, the series' datapoints slice will be moved back to the pool. +* 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. @@ -72,7 +72,7 @@ for now we assume that multi-steps in a row is not that common, and COW seems mo This leaves the problem of effectively managing allocations and using a sync.Pool. -Note that the expr library can be called by different clients. At this point only Metrictank uses it, but we intend this lirbrary to be easily embeddable in other programs. +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. diff --git a/docs/render-path.md b/docs/render-path.md index 9f3621863b..58d53c2c9e 100644 --- a/docs/render-path.md +++ b/docs/render-path.md @@ -113,7 +113,7 @@ First, let's look at some definitions. 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 simplicty, we don't implement this and treat all IA functions as functions that may change the interval of series in unpredicatable ways. +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 @@ -133,9 +133,16 @@ Generally, if series have different intervals, they can keep those and we return 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 wen we know the normalization won't affect anything else. -This is the case when series go from fetching to transparent aggregation, possibly with some processing functions - except opaque aggregation(s) or IA-function(s) - in between, and -with asPercent in a certain mode (where it has to normalize all inputs), but not with divideSeries where it applies the same divisor to multiple dividend inputs, for example. +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: ``` @@ -152,13 +159,12 @@ Likewise, if the query is `groupByNode(group(A,B), 2, callback='sum')` we cannot 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, such that +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. But this is an edge case -2) uses slightly more of the chunk cache. +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 diff --git a/expr/plan.go b/expr/plan.go index 1392ce64bd..bbf30760ba 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -16,9 +16,11 @@ type Optimizations struct { } func (o Optimizations) ApplyUserPrefs(s string) Optimizations { + // no user override. stick to what we have if s == "" { return o } + // 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" { diff --git a/util/combinations.go b/util/combinations.go index 88a225d3d6..d5125ae689 100644 --- a/util/combinations.go +++ b/util/combinations.go @@ -11,31 +11,31 @@ func AllCombinationsUint32(parts [][]uint32) (out [][]uint32) { out = make([][]uint32, 0, num) // will contain idx of which one to pick for each part - idexes := make([]int, len(parts)) + indexes := make([]int, len(parts)) mainloop: for { - // update idexes: + // 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(idexes) - 1; i >= 0; i-- { - if idexes[i] >= len(parts[i]) { + for i := len(indexes) - 1; i >= 0; i-- { + if indexes[i] >= len(parts[i]) { if i == 0 { break mainloop } - idexes[i] = 0 - idexes[i-1]++ + indexes[i] = 0 + indexes[i-1]++ } } combo := make([]uint32, len(parts)) for i, part := range parts { - combo[i] = part[idexes[i]] + combo[i] = part[indexes[i]] } out = append(out, combo) // always bump idx of the last one - idexes[len(parts)-1]++ + indexes[len(parts)-1]++ } return out } From d5c422836348989480d726b95a6c34f6357a7b0b Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 24 Jan 2020 19:00:04 +0100 Subject: [PATCH 43/46] asPercent safe again --- expr/funcs.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/expr/funcs.go b/expr/funcs.go index 82e66dfc5f..ff02d96e61 100644 --- a/expr/funcs.go +++ b/expr/funcs.go @@ -57,7 +57,7 @@ func init() { "aliasByTags": {NewAliasByNode, true}, "aliasByNode": {NewAliasByNode, true}, "aliasSub": {NewAliasSub, true}, - "asPercent": {NewAsPercent, false}, // disabled because it needs an update for runtime normalization and possibly MDP/pre-normalization + "asPercent": {NewAsPercent, true}, "avg": {NewAggregateConstructor("average", crossSeriesAvg), true}, "averageAbove": {NewFilterSeriesConstructor("average", ">"), true}, "averageBelow": {NewFilterSeriesConstructor("average", "<="), true}, From 7198dfb7ef61c27d44bc3b4f41ea1a860c3c3bc9 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 24 Jan 2020 19:24:37 +0100 Subject: [PATCH 44/46] robert feedback --- api/graphite.go | 12 ++++++++++-- api/graphite_req.go | 9 ++++++++- api/models/request.go | 6 ++++-- devdocs/render-request-handling.md | 8 ++++---- expr/func_divideseries.go | 2 +- expr/plan.go | 16 +++++++++------- 6 files changed, 36 insertions(+), 17 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index a414a4600f..e10ceb8343 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -226,7 +226,11 @@ func (s *Server) renderMetrics(ctx *middleware.Context, request models.GraphiteR mdp = 0 } - opts := optimizations.ApplyUserPrefs(request.Optimizations) + 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 { @@ -1422,7 +1426,11 @@ func (s *Server) showPlan(ctx *middleware.Context, request models.GraphiteRender stable := request.Process == "stable" mdp := request.MaxDataPoints - opts := optimizations.ApplyUserPrefs(request.Optimizations) + 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())) diff --git a/api/graphite_req.go b/api/graphite_req.go index 282cabb163..5dbb7bbc3d 100644 --- a/api/graphite_req.go +++ b/api/graphite_req.go @@ -21,6 +21,7 @@ func NewReqMap() *ReqMap { } } +// Add adds a models.Req to the ReqMap func (r *ReqMap) Add(req models.Req) { r.cnt++ if req.PNGroup == 0 { @@ -29,6 +30,8 @@ func (r *ReqMap) Add(req models.Req) { } 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" @@ -51,13 +54,14 @@ type PNGroupSplit struct { mdpno []models.Req // not MDP-optimizable reqs } -// ReqsPlan holds requests that have been planned +// 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), @@ -84,6 +88,7 @@ func NewReqsPlan(reqs ReqMap) ReqsPlan { 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 { @@ -103,6 +108,7 @@ func (rp ReqsPlan) PointsFetch() uint32 { 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" @@ -148,6 +154,7 @@ func (rp ReqsPlan) PointsReturn(planMDP uint32) uint32 { 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...) diff --git a/api/models/request.go b/api/models/request.go index ed844bbc7e..9c3b98443c 100644 --- a/api/models/request.go +++ b/api/models/request.go @@ -52,7 +52,7 @@ type Req struct { // 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 minus the ones that need request planning +// 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, @@ -72,7 +72,7 @@ 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 minus the ones that need request planning +// 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() @@ -97,6 +97,7 @@ func (r *Req) Plan(i int, ret conf.Retention) { 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 @@ -130,6 +131,7 @@ func (r *Req) AdjustTo(interval, from uint32, rets []conf.Retention) { 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 } diff --git a/devdocs/render-request-handling.md b/devdocs/render-request-handling.md index f2270e385d..fc458c4b4d 100644 --- a/devdocs/render-request-handling.md +++ b/devdocs/render-request-handling.md @@ -21,16 +21,16 @@ ## MDP-optimization -MDP at the leaf of the expr tree (fetch request) 0 means don't optimize, set it to >0 means, can be optimized. +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 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 to MDP to 0 on the context (and thus also on any subsequent requests) +* 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 ) +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 diff --git a/expr/func_divideseries.go b/expr/func_divideseries.go index 3d823a8bd4..e8eb2a7aeb 100644 --- a/expr/func_divideseries.go +++ b/expr/func_divideseries.go @@ -61,7 +61,7 @@ func (s *FuncDivideSeries) Exec(cache map[Req][]models.Series) ([]models.Series, // 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 the require no special attention) + // (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 { diff --git a/expr/plan.go b/expr/plan.go index bbf30760ba..5e24be7369 100644 --- a/expr/plan.go +++ b/expr/plan.go @@ -15,27 +15,29 @@ type Optimizations struct { MDP bool } -func (o Optimizations) ApplyUserPrefs(s string) Optimizations { +func (o Optimizations) ApplyUserPrefs(s string) (Optimizations, error) { // no user override. stick to what we have if s == "" { - return o + 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 + return o, nil } prefs := strings.Split(s, ",") for _, pref := range prefs { - if pref == "pn" { + switch pref { + case "pn": o.PreNormalization = true - } - if pref == "mdp" { + case "mdp": o.MDP = true + default: + return o, fmt.Errorf("unrecognized optimization %q", pref) } } - return o + return o, nil } // Req represents a request for one/more series From 260c7eb4b0a8359cd3fa52f6945fac1a423ff66d Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Fri, 24 Jan 2020 19:31:28 +0100 Subject: [PATCH 45/46] sean feedback 2 --- api/query_engine.go | 21 +++++++++++++++++---- devdocs/render-request-handling.md | 2 +- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/api/query_engine.go b/api/query_engine.go index 1480b3feff..719c541753 100644 --- a/api/query_engine.go +++ b/api/query_engine.go @@ -258,13 +258,20 @@ func planLowestResForMDPMulti(now, from, to uint32, reqs []models.Req) ([]models var validIntervalss [][]uint32 - for i := range reqs { - req := &reqs[i] + // 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 := getRetentions(*req) + rets := mdata.Schemas.Get(schemaID).Retentions.Rets var validIntervals []uint32 for _, ret := range rets { - if ret.Ready <= from && req.TTL >= minTTL { + if ret.Ready <= from && ret.MaxRetention() >= int(minTTL) { ok = true validIntervals = append(validIntervals, uint32(ret.SecondsPerPoint)) } @@ -285,6 +292,12 @@ func planLowestResForMDPMulti(now, from, to uint32, reqs []models.Req) ([]models } } + // 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 diff --git a/devdocs/render-request-handling.md b/devdocs/render-request-handling.md index fc458c4b4d..75acc139cb 100644 --- a/devdocs/render-request-handling.md +++ b/devdocs/render-request-handling.md @@ -12,7 +12,7 @@ * 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. - * `planRequests`: this plans at all models.Req objects, which means decide which archive to read from, whether to apply normalization, etc + * `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 From 2f9369ea1efaed334fd962342a1690a23e835881 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Tue, 28 Jan 2020 17:29:15 +0200 Subject: [PATCH 46/46] planRequests() erroring is a user error --- api/graphite.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/api/graphite.go b/api/graphite.go index e10ceb8343..460bd85060 100644 --- a/api/graphite.go +++ b/api/graphite.go @@ -756,14 +756,13 @@ func (s *Server) executePlan(ctx context.Context, orgId uint32, plan expr.Plan) var err error var rp *ReqsPlan rp, err = planRequests(uint32(time.Now().Unix()), minFrom, maxTo, reqs, plan.MaxDataPoints, maxPointsPerReqSoft, maxPointsPerReqHard) + if err != nil { + return nil, meta, err + } meta.RenderStats.PointsFetch = rp.PointsFetch() meta.RenderStats.PointsReturn = rp.PointsReturn(plan.MaxDataPoints) reqsList := rp.List() - if err != nil { - log.Errorf("HTTP Render planRequests error: %s", err.Error()) - return nil, meta, err - } span := opentracing.SpanFromContext(ctx) span.SetTag("num_reqs", len(reqsList)) span.SetTag("points_fetch", meta.RenderStats.PointsFetch)