From b209daef62dfd01e3cda2bcbf701409a0d5a4dd1 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Sat, 17 Jun 2017 00:51:11 +0000 Subject: [PATCH 01/17] Adds a write buffer to reorder data on the fly This works by delaying the compression of incoming data until the size of the buffered data exceeds the "reorder window", then it gets compressed as usual. The buffer is configurable on a per-pattern basis, together with the aggregations. --- api/dataprocessor.go | 39 ++--- api/dataprocessor_test.go | 8 +- conf/aggregations.go | 38 +++++ mdata/aggmetric.go | 66 ++++++-- mdata/aggmetric_test.go | 90 ++++++++++- mdata/aggregator_test.go | 4 +- mdata/ifaces.go | 11 +- mdata/init.go | 11 +- mdata/write_buffer.go | 155 ++++++++++++++++++ mdata/write_buffer_test.go | 322 +++++++++++++++++++++++++++++++++++++ usage/usage_test.go | 8 +- 11 files changed, 701 insertions(+), 51 deletions(-) create mode 100644 mdata/write_buffer.go create mode 100644 mdata/write_buffer_test.go diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 586982b2d0..7117423de0 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -10,6 +10,7 @@ import ( "github.com/raintank/metrictank/api/models" "github.com/raintank/metrictank/consolidation" + "github.com/raintank/metrictank/mdata" "github.com/raintank/metrictank/mdata/chunk" "github.com/raintank/metrictank/util" "github.com/raintank/worldping-api/pkg/log" @@ -314,26 +315,27 @@ func AggMetricKey(key, archive string, aggSpan uint32) string { func (s *Server) getSeriesFixed(req models.Req, consolidator consolidation.Consolidator) []schema.Point { ctx := newRequestContext(&req, consolidator) - iters := s.getSeries(ctx) - points := s.itersToPoints(ctx, iters) - return Fix(points, req.From, req.To, req.ArchInterval) + res := s.getSeries(ctx) + res.Raw = append(s.itersToPoints(ctx, res.Iters), res.Raw...) + return Fix(res.Raw, req.From, req.To, req.ArchInterval) } -func (s *Server) getSeries(ctx *requestContext) []chunk.Iter { +func (s *Server) getSeries(ctx *requestContext) mdata.MetricResult { - oldest, memIters := s.getSeriesAggMetrics(ctx) - log.Debug("oldest from aggmetrics is %d", oldest) + res := s.getSeriesAggMetrics(ctx) + log.Debug("oldest from aggmetrics is %d", res.Oldest) - if oldest <= ctx.From { + if res.Oldest <= ctx.From { reqSpanMem.ValueUint32(ctx.To - ctx.From) - return memIters + return res } // if oldest < to -> search until oldest, we already have the rest from mem // if to < oldest -> no need to search until oldest, only search until to - until := util.Min(oldest, ctx.To) + until := util.Min(res.Oldest, ctx.To) - return append(s.getSeriesCachedStore(ctx, until), memIters...) + res.Iters = append(s.getSeriesCachedStore(ctx, until), res.Iters...) + return res } // getSeries returns points from mem (and cassandra if needed), within the range from (inclusive) - to (exclusive) @@ -363,24 +365,23 @@ func (s *Server) itersToPoints(ctx *requestContext, iters []chunk.Iter) []schema return points } -func (s *Server) getSeriesAggMetrics(ctx *requestContext) (uint32, []chunk.Iter) { - oldest := ctx.Req.To - var memIters []chunk.Iter - +func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.MetricResult { metric, ok := s.MemoryStore.Get(ctx.Key) if !ok { - return oldest, memIters + return mdata.MetricResult{ + Oldest: ctx.Req.To, + Iters: make([]chunk.Iter, 0), + Raw: make([]schema.Point, 0), + } } if ctx.Cons != consolidation.None { logLoad("memory", ctx.AggKey, ctx.From, ctx.To) - oldest, memIters = metric.GetAggregated(ctx.Cons, ctx.Req.ArchInterval, ctx.From, ctx.To) + return metric.GetAggregated(ctx.Cons, ctx.Req.ArchInterval, ctx.From, ctx.To) } else { logLoad("memory", ctx.Req.Key, ctx.From, ctx.To) - oldest, memIters = metric.Get(ctx.From, ctx.To) + return metric.Get(ctx.From, ctx.To) } - - return oldest, memIters } // will only fetch until until, but uses ctx.To for debug logging diff --git a/api/dataprocessor_test.go b/api/dataprocessor_test.go index 4e9025b94d..27de9c33c4 100644 --- a/api/dataprocessor_test.go +++ b/api/dataprocessor_test.go @@ -767,11 +767,11 @@ func TestGetSeriesAggMetrics(t *testing.T) { metric.Add(i, float64(i^2)) } - oldest, iters := srv.getSeriesAggMetrics(ctx) + res := srv.getSeriesAggMetrics(ctx) timestamps := make([]uint32, 0) values := make([]float64, 0) - for _, it := range iters { + for _, it := range res.Iters { for it.Next() { ts, val := it.Values() timestamps = append(timestamps, ts) @@ -782,8 +782,8 @@ func TestGetSeriesAggMetrics(t *testing.T) { // should be the T0 of the chunk from (1744) is in // 1744 - (1744 % 600) = 1200 expected := uint32(1200) - if oldest != expected { - t.Errorf("Expected oldest to be %d but got %d", expected, oldest) + if res.Oldest != expected { + t.Errorf("Expected oldest to be %d but got %d", expected, res.Oldest) } // number of returned ts should be the number of chunks the searched range spans across * chunkspan diff --git a/conf/aggregations.go b/conf/aggregations.go index a6be36eaea..954527fb2d 100644 --- a/conf/aggregations.go +++ b/conf/aggregations.go @@ -9,6 +9,11 @@ import ( "github.com/alyu/configparser" ) +type WriteBufferConf struct { + ReorderWindow uint32 + FlushMin uint32 +} + // Aggregations holds the aggregation definitions type Aggregations struct { Data []Aggregation @@ -20,6 +25,7 @@ type Aggregation struct { Pattern *regexp.Regexp XFilesFactor float64 AggregationMethod []Method + WriteBufferConf *WriteBufferConf } // NewAggregations create instance of Aggregations @@ -85,6 +91,38 @@ func ReadAggregations(file string) (Aggregations, error) { } } + writeBufferStr := s.ValueOf("writeBuffer") + if len(writeBufferStr) > 0 { + writeBufferStrs := strings.Split(writeBufferStr, ",") + if len(writeBufferStrs) != 2 { + err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 parts: %s", item.Name, writeBufferStr) + return Aggregations{}, err + } + + reorderWindow, err := strconv.ParseUint(writeBufferStrs[0], 10, 32) + if err != nil { + err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 numbers: %s", item.Name, writeBufferStr) + return Aggregations{}, err + } + flushMin, err := strconv.ParseUint(writeBufferStrs[1], 10, 32) + if err != nil { + err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 numbers: %s", item.Name, writeBufferStr) + return Aggregations{}, err + } + if flushMin < 1 && reorderWindow > 0 { + err = fmt.Errorf("[%s]: Failed to parse write buffer conf, flush minimum needs to be > 0: %s", item.Name, writeBufferStr) + return Aggregations{}, err + + } + // if reorderWindow == 0 we just disable the buffer + if reorderWindow > 0 { + item.WriteBufferConf = &WriteBufferConf{ + ReorderWindow: uint32(reorderWindow), + FlushMin: uint32(flushMin), + } + } + } + result.Data = append(result.Data, item) } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index ebe0d5ea1f..c7d717c587 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -12,6 +12,8 @@ import ( "github.com/raintank/metrictank/mdata/cache" "github.com/raintank/metrictank/mdata/chunk" "github.com/raintank/worldping-api/pkg/log" + + "gopkg.in/raintank/schema.v1" ) // AggMetric takes in new values, updates the in-memory data and streams the points to aggregators @@ -26,6 +28,7 @@ type AggMetric struct { cachePusher cache.CachePusher sync.RWMutex Key string + wb *WriteBuffer CurrentChunkPos int // element in []Chunks that is active. All others are either finished or nil. NumChunks uint32 // max size of the circular buffer ChunkSpan uint32 // span of individual chunks in seconds @@ -61,6 +64,10 @@ func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retent // garbage collected right after creating it, before we can push to it. lastWrite: uint32(time.Now().Unix()), } + if agg != nil && agg.WriteBufferConf != nil { + m.wb = NewWriteBuffer(agg.WriteBufferConf, m.add) + } + for _, ret := range retentions[1:] { m.aggregators = append(m.aggregators, NewAggregator(store, cachePusher, key, ret, *agg, dropFirstChunk)) } @@ -132,7 +139,7 @@ func (a *AggMetric) getChunk(pos int) *chunk.Chunk { return a.Chunks[pos] } -func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) (uint32, []chunk.Iter) { +func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) MetricResult { // no lock needed cause aggregators don't change at runtime for _, a := range a.aggregators { if a.span == aggSpan { @@ -161,7 +168,7 @@ func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSp // Get all data between the requested time ranges. From is inclusive, to is exclusive. from <= x < to // more data then what's requested may be included // also returns oldest point we have, so that if your query needs data before it, the caller knows when to query cassandra -func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { +func (a *AggMetric) Get(from, to uint32) MetricResult { pre := time.Now() if LogLevel < 2 { log.Debug("AM %s Get(): %d - %d (%s - %s) span:%ds", a.Key, from, to, TS(from), TS(to), to-from-1) @@ -172,12 +179,28 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { a.RLock() defer a.RUnlock() + result := MetricResult{ + Oldest: math.MaxInt32, + Iters: make([]chunk.Iter, 0), + Raw: make([]schema.Point, 0), + } + + if a.wb != nil { + result.Raw = a.wb.Get() + if len(result.Raw) > 0 { + result.Oldest = result.Raw[0].Ts + if result.Oldest <= from { + return result + } + } + } + if len(a.Chunks) == 0 { // we dont have any data yet. if LogLevel < 2 { log.Debug("AM %s Get(): no data for requested range.", a.Key) } - return math.MaxInt32, make([]chunk.Iter, 0) + return result } newestChunk := a.getChunk(a.CurrentChunkPos) @@ -197,7 +220,8 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { if LogLevel < 2 { log.Debug("AM %s Get(): no data for requested range.", a.Key) } - return from, make([]chunk.Iter, 0) + result.Oldest = from + return result } // get the oldest chunk we have. @@ -219,7 +243,7 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { oldestChunk := a.getChunk(oldestPos) if oldestChunk == nil { log.Error(3, "unexpected nil chunk.") - return math.MaxInt32, make([]chunk.Iter, 0) + return result } // The first chunk is likely only a partial chunk. If we are not the primary node @@ -233,7 +257,7 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { oldestChunk = a.getChunk(oldestPos) if oldestChunk == nil { log.Error(3, "unexpected nil chunk.") - return math.MaxInt32, make([]chunk.Iter, 0) + return result } } @@ -242,7 +266,8 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { if LogLevel < 2 { log.Debug("AM %s Get(): no data for requested range", a.Key) } - return oldestChunk.T0, make([]chunk.Iter, 0) + result.Oldest = oldestChunk.T0 + return result } // Find the oldest Chunk that the "from" ts falls in. If from extends before the oldest @@ -255,7 +280,8 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { oldestChunk = a.getChunk(oldestPos) if oldestChunk == nil { log.Error(3, "unexpected nil chunk.") - return to, make([]chunk.Iter, 0) + result.Oldest = to + return result } } @@ -274,15 +300,15 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { newestChunk = a.getChunk(newestPos) if newestChunk == nil { log.Error(3, "unexpected nil chunk.") - return to, make([]chunk.Iter, 0) + result.Oldest = to + return result } } // now just start at oldestPos and move through the Chunks circular Buffer to newestPos - iters := make([]chunk.Iter, 0, a.NumChunks) for { c := a.getChunk(oldestPos) - iters = append(iters, chunk.NewIter(c.Iter())) + result.Iters = append(result.Iters, chunk.NewIter(c.Iter())) if oldestPos == newestPos { break @@ -295,7 +321,8 @@ func (a *AggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { } memToIterDuration.Value(time.Now().Sub(pre)) - return oldestChunk.T0, iters + result.Oldest = oldestChunk.T0 + return result } // this function must only be called while holding the lock @@ -402,11 +429,24 @@ func (a *AggMetric) persist(pos int) { return } -// don't ever call with a ts of 0, cause we use 0 to mean not initialized! func (a *AggMetric) Add(ts uint32, val float64) { a.Lock() defer a.Unlock() + if a.wb == nil { + // write directly + a.add(ts, val) + } else { + // write through write buffer, returns false if ts is out of reorder window + if !a.wb.Add(ts, val) { + metricsTooOld.Inc() + } + } +} + +// don't ever call with a ts of 0, cause we use 0 to mean not initialized! +// assumes a write lock is held by the call-site +func (a *AggMetric) add(ts uint32, val float64) { t0 := ts - (ts % a.ChunkSpan) if len(a.Chunks) == 0 { diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index 849269a733..b0adb85120 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -2,6 +2,8 @@ package mdata import ( "fmt" + "regexp" + "sort" "testing" "time" @@ -17,6 +19,12 @@ type point struct { val float64 } +type points []point + +func (a points) Len() int { return len(a) } +func (a points) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a points) Less(i, j int) bool { return a[i].ts < a[j].ts } + func (p point) String() string { return fmt.Sprintf("point{%0.f at %d}", p.val, p.ts) } @@ -37,13 +45,28 @@ func (c *Checker) Add(ts uint32, val float64) { c.points = append(c.points, point{ts, val}) } +func (c *Checker) DropPointByTs(ts uint32) { + i := 0 + for { + if i == len(c.points) { + return + } + if c.points[i].ts == ts { + c.points = append(c.points[:i], c.points[i+1:]...) + } else { + i++ + } + } +} + // from to is the range that gets requested from AggMetric // first/last is what we use as data range to compare to (both inclusive) // these may be different because AggMetric returns broader rangers (due to packed format), func (c *Checker) Verify(primary bool, from, to, first, last uint32) { currentClusterStatus := cluster.Manager.IsPrimary() + sort.Sort(points(c.points)) cluster.Manager.SetPrimary(primary) - _, iters := c.agg.Get(from, to) + res := c.agg.Get(from, to) // we don't do checking or fancy logic, it is assumed that the caller made sure first and last are ts of actual points var pi int // index of first point we want var pj int // index of last point we want @@ -53,18 +76,27 @@ func (c *Checker) Verify(primary bool, from, to, first, last uint32) { } c.t.Logf("verifying AggMetric.Get(%d,%d) =?= %d <= ts <= %d", from, to, first, last) index := pi - 1 - for _, iter := range iters { + for _, iter := range res.Iters { for iter.Next() { index++ tt, vv := iter.Values() if index > pj { - c.t.Fatalf("Values()=(%v,%v), want end of stream\n", tt, vv) + c.t.Fatalf("Iters: Values()=(%v,%v), want end of stream\n", tt, vv) } if c.points[index].ts != tt || c.points[index].val != vv { - c.t.Fatalf("Values()=(%v,%v), want (%v,%v)\n", tt, vv, c.points[index].ts, c.points[index].val) + c.t.Fatalf("Iters: Values()=(%v,%v), want (%v,%v)\n", tt, vv, c.points[index].ts, c.points[index].val) } } } + for _, point := range res.Raw { + index++ + if index > pj { + c.t.Fatalf("Raw: Values()=(%v,%v), want end of stream\n", point.Ts, point.Val) + } + if c.points[index].ts != point.Ts || c.points[index].val != point.Val { + c.t.Fatalf("Raw: Values()=(%v,%v), want (%v,%v)\n", point.Ts, point.Val, c.points[index].ts, c.points[index].val) + } + } if index != pj { c.t.Fatalf("not all values returned. missing %v", c.points[index:pj+1]) } @@ -199,6 +231,56 @@ func TestAggMetric(t *testing.T) { // c.Verify(true, 800, 1299, 1299, 1299) } +func TestAggMetricWithWriteBuffer(t *testing.T) { + cluster.Init("default", "test", time.Now(), "http", 6060) + + agg := conf.Aggregation{ + Name: "Default", + Pattern: regexp.MustCompile(".*"), + XFilesFactor: 0.5, + AggregationMethod: []conf.Method{conf.Avg}, + WriteBufferConf: &conf.WriteBufferConf{ + ReorderWindow: 10, + FlushMin: 10, + }, + } + ret := []conf.Retention{conf.NewRetentionMT(1, 1, 100, 5, true)} + c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, &agg, false)) + + // basic case, single range + c.Add(101, 101) + c.Verify(true, 100, 200, 101, 101) + c.Add(105, 105) + c.Verify(true, 100, 199, 101, 105) + c.Add(115, 115) + c.Add(125, 125) + c.Add(135, 135) + c.Verify(true, 100, 199, 101, 135) + + // add new ranges, aligned and unaligned + c.Add(200, 200) + c.Add(315, 315) + c.Verify(true, 100, 399, 101, 315) + + metricsTooOld.SetUint32(0) + + // adds 14 entries that are out of order and the write buffer should order the first 13 + // including the previous 7 it will then reach 20 which is = reorder window + flush min, so it causes a flush + // the last item (14th) will be added out of order, after the buffer is flushed, so it increases metricsTooOld + for i := uint32(314); i > 300; i-- { + c.Add(i, float64(i)) + } + c.DropPointByTs(301) + + // get subranges + c.Verify(true, 100, 320, 101, 315) + + // one point has been added out of order and too old for the buffer to reorder + if metricsTooOld.Peek() != 1 { + t.Fatalf("Expected the out off order count to be 1") + } +} + func TestAggMetricDropFirstChunk(t *testing.T) { cluster.Init("default", "test", time.Now(), "http", 6060) cluster.Manager.SetPrimary(true) diff --git a/mdata/aggregator_test.go b/mdata/aggregator_test.go index 91db808382..dff6a73fcc 100644 --- a/mdata/aggregator_test.go +++ b/mdata/aggregator_test.go @@ -44,9 +44,9 @@ func TestAggregator(t *testing.T) { cluster.Init("default", "test", time.Now(), "http", 6060) compare := func(key string, metric Metric, expected []schema.Point) { cluster.Manager.SetPrimary(true) - _, iters := metric.Get(0, 1000) + res := metric.Get(0, 1000) got := make([]schema.Point, 0, len(expected)) - for _, iter := range iters { + for _, iter := range res.Iters { for iter.Next() { ts, val := iter.Values() got = append(got, schema.Point{Val: val, Ts: ts}) diff --git a/mdata/ifaces.go b/mdata/ifaces.go index 823124f9f5..4e8c02d96a 100644 --- a/mdata/ifaces.go +++ b/mdata/ifaces.go @@ -3,6 +3,7 @@ package mdata import ( "github.com/raintank/metrictank/consolidation" "github.com/raintank/metrictank/mdata/chunk" + "gopkg.in/raintank/schema.v1" ) type Metrics interface { @@ -12,6 +13,12 @@ type Metrics interface { type Metric interface { Add(ts uint32, val float64) - Get(from, to uint32) (uint32, []chunk.Iter) - GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) (uint32, []chunk.Iter) + Get(from, to uint32) MetricResult + GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) MetricResult +} + +type MetricResult struct { + Raw []schema.Point + Iters []chunk.Iter + Oldest uint32 } diff --git a/mdata/init.go b/mdata/init.go index ca75a68ba2..1a924c3492 100644 --- a/mdata/init.go +++ b/mdata/init.go @@ -22,9 +22,14 @@ var ( // metric tank.chunk_operations.clear is a counter of how many chunks are cleared (replaced by new chunks) chunkClear = stats.NewCounter32("tank.chunk_operations.clear") - // metric tank.metrics_too_old is points that go back in time. - // E.g. for any given series, when a point has a timestamp - // that is not higher than the timestamp of the last written timestamp for that series. + // metric tank.metrics_reordered is the number of points received that are going back in time, but are still + // within the reorder window. in such a case they will be inserted in the correct order. + // E.g. if the reorder window is 60 (datapoints) then points may be inserted at random order as long as their + // ts is not older than the 60th datapoint counting from the newest. + metricsReordered = stats.NewCounter32("tank.metrics_reorderd") + + // metric tank.metrics_too_old is points that go back in time beyond the scope of the reorder window. + // these points will end up being dropped and lost. metricsTooOld = stats.NewCounter32("tank.metrics_too_old") // metric tank.add_to_closed_chunk is points received for the most recent chunk diff --git a/mdata/write_buffer.go b/mdata/write_buffer.go new file mode 100644 index 0000000000..226339a1f9 --- /dev/null +++ b/mdata/write_buffer.go @@ -0,0 +1,155 @@ +package mdata + +import ( + "fmt" + "sync" + + "github.com/raintank/metrictank/conf" + "gopkg.in/raintank/schema.v1" +) + +var bufPool = sync.Pool{New: func() interface{} { return &entry{} }} + +/* + * The write buffer keeps a window of data during which it is ok to send data out of order. + * Once the reorder window plus flush minimum has passed it will try to flush the data out. + * The write buffer itself is not thread safe because it is only used by AggMetric, + * which is thread safe, so there is no locking in the buffer. + */ + +type WriteBuffer struct { + reorderWindow uint32 // window size in datapoints during which out of order is allowed + len uint32 // number of datapoints in the buffer + lastFlush uint32 // the timestamp of the last point that's been flushed + flushMin uint32 // min number of datapoints to flush at once + first *entry // first buffer entry + last *entry // last buffer entry + flush func(uint32, float64) // flushing callback +} + +type entry struct { + ts uint32 + val float64 + next, prev *entry +} + +func NewWriteBuffer(conf *conf.WriteBufferConf, flush func(uint32, float64)) *WriteBuffer { + return &WriteBuffer{ + reorderWindow: conf.ReorderWindow, + flushMin: conf.FlushMin, + flush: flush, + } +} + +func (wb *WriteBuffer) Add(ts uint32, val float64) bool { + // out of order and too old + if ts < wb.lastFlush { + return false + } + + e := bufPool.Get().(*entry) + e.ts = ts + e.val = val + + // initializing the linked list + if wb.first == nil { + e.next = nil + e.prev = nil + wb.first = e + wb.last = e + wb.len++ + } else { + if ts < wb.last.ts { + metricsReordered.Inc() + } + + // in the normal case data should be added in order, so this will only iterate once + for i := wb.last; i != nil; i = i.prev { + if ts > i.ts { + if i.next == nil { + wb.last = e + } else { + i.next.prev = e + } + e.next = i.next + e.prev = i + i.next = e + e = nil + wb.len++ + break + } + // overwrite value + if ts == i.ts { + i.val = val + e = nil + break + } + } + if e != nil { + // unlikely case where the added entry is the oldest one present + e.prev = nil + e.next = wb.first + wb.first.prev = e + wb.first = e + wb.len++ + } + } + + wb.flushIfReady() + + return true +} + +// if buffer is ready for flushing, this will flush it +func (wb *WriteBuffer) flushIfReady() { + // not enough data, not ready to flush + if wb.len < wb.flushMin+wb.reorderWindow { + return + } + + // we want to flush until the length is equal to the reorder window + flushCount := wb.len - wb.reorderWindow + + nextEntry := wb.first + for i := uint32(0); i < flushCount; i++ { + flushEntry := nextEntry + nextEntry = flushEntry.next + wb.flush(flushEntry.ts, flushEntry.val) + bufPool.Put(flushEntry) + } + + wb.len = wb.reorderWindow + wb.first = nextEntry + wb.first.prev = nil + wb.lastFlush = wb.first.ts +} + +// returns a formatted string that shows the current buffer content, +// only used for debugging purposes and should never be called in prod +func (wb *WriteBuffer) formatted() string { + var str string + var id int + str = fmt.Sprintf("Buffer len: %d first: %p last: %p \n", wb.len, wb.first, wb.last) + for i := wb.first; i != nil; i = i.next { + str = fmt.Sprintf( + "%sId: %d ts: %d val: %f addr: %p prev: %p next: %p\n", + str, id, i.ts, i.val, i, i.prev, i.next, + ) + id++ + } + return str +} + +// returns all the data in the buffer as a raw list of points +func (wb *WriteBuffer) Get() []schema.Point { + res := make([]schema.Point, 0, wb.len) + if wb.first == nil { + return res + } + + for i := wb.first; i != nil; i = i.next { + res = append(res, schema.Point{Val: i.val, Ts: i.ts}) + } + + return res +} diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go new file mode 100644 index 0000000000..9d085bfe03 --- /dev/null +++ b/mdata/write_buffer_test.go @@ -0,0 +1,322 @@ +package mdata + +import ( + "fmt" + "testing" + + "github.com/raintank/metrictank/conf" + "gopkg.in/raintank/schema.v1" +) + +func testAddAndGet(t *testing.T, conf *conf.WriteBufferConf, collector func(uint32, float64), testData, expectedData []schema.Point, expectAddFail bool) { + b := NewWriteBuffer(conf, collector) + gotFailure := false + for _, point := range testData { + success := b.Add(point.Ts, point.Val) + if !success { + gotFailure = true + } + b.flushIfReady() + } + if expectAddFail && !gotFailure { + t.Fatal("Expected an add to fail, but they all succeeded") + } + returned := b.Get() + + if len(expectedData) != len(returned) { + t.Fatal("Length of returned and testData data unequal") + } + if !pointSlicesAreEqual(expectedData, returned) { + t.Fatal(fmt.Sprintf("Returned data does not match testData data %+v, %+v", testData, returned)) + } +} + +func pointSlicesAreEqual(a, b []schema.Point) bool { + if len(a) != len(b) { + return false + } + for i := range a { + if (a[i].Ts != b[i].Ts) || (a[i].Val != b[i].Val) { + return false + } + } + return true +} + +func unsort(data []schema.Point, unsortBy int) []schema.Point { + out := make([]schema.Point, len(data)) + i := 0 + for ; i < len(data)-unsortBy; i = i + unsortBy { + for j := 0; j < unsortBy; j++ { + out[i+j] = data[i+unsortBy-j-1] + } + } + for ; i < len(data); i++ { + out[i] = data[i] + } + return out +} + +func TestUnsort(t *testing.T) { + testData := []schema.Point{ + {Ts: 0, Val: 0}, + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 3, Val: 300}, + {Ts: 4, Val: 400}, + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 7, Val: 700}, + {Ts: 8, Val: 800}, + {Ts: 9, Val: 900}, + } + expectedData := []schema.Point{ + {Ts: 2, Val: 200}, + {Ts: 1, Val: 100}, + {Ts: 0, Val: 0}, + {Ts: 5, Val: 500}, + {Ts: 4, Val: 400}, + {Ts: 3, Val: 300}, + {Ts: 8, Val: 800}, + {Ts: 7, Val: 700}, + {Ts: 6, Val: 600}, + {Ts: 9, Val: 900}, + } + unsortedData := unsort(testData, 3) + + for i := 0; i < len(expectedData); i++ { + if unsortedData[i] != expectedData[i] { + t.Fatalf("unsort function returned unexpected data %+v", unsortedData) + } + } +} + +func TestAddAndGetInOrder(t *testing.T) { + testData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 3, Val: 300}, + } + expectedData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 3, Val: 300}, + } + conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} + testAddAndGet(t, conf, nil, testData, expectedData, false) +} + +func TestAddAndGetInReverseOrderOutOfWindow(t *testing.T) { + testData := []schema.Point{ + {Ts: 3, Val: 300}, + {Ts: 2, Val: 200}, + {Ts: 1, Val: 100}, + } + expectedData := []schema.Point{ + {Ts: 3, Val: 300}, + } + conf := &conf.WriteBufferConf{ReorderWindow: 1, FlushMin: 1} + collector := func(ts uint32, val float64) {} + testAddAndGet(t, conf, collector, testData, expectedData, true) +} + +func TestAddAndGetOutOfOrderInsideWindow(t *testing.T) { + testData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 4, Val: 400}, + {Ts: 3, Val: 300}, + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 8, Val: 800}, + {Ts: 7, Val: 700}, + {Ts: 9, Val: 900}, + } + expectedData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 3, Val: 300}, + {Ts: 4, Val: 400}, + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 7, Val: 700}, + {Ts: 8, Val: 800}, + {Ts: 9, Val: 900}, + } + conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} + testAddAndGet(t, conf, nil, testData, expectedData, false) +} + +func TestAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { + testData := []schema.Point{ + {Ts: 2, Val: 200}, + {Ts: 4, Val: 400}, + {Ts: 3, Val: 300}, + {Ts: 5, Val: 500}, + {Ts: 1, Val: 100}, + {Ts: 6, Val: 600}, + {Ts: 8, Val: 800}, + {Ts: 7, Val: 700}, + {Ts: 9, Val: 900}, + } + expectedData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 2, Val: 200}, + {Ts: 3, Val: 300}, + {Ts: 4, Val: 400}, + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 7, Val: 700}, + {Ts: 8, Val: 800}, + {Ts: 9, Val: 900}, + } + conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} + testAddAndGet(t, conf, nil, testData, expectedData, false) +} + +func TestOmitFlushIfNotEnoughData(t *testing.T) { + conf := &conf.WriteBufferConf{ReorderWindow: 9, FlushMin: 1} + collector := func(ts uint32, val float64) { + t.Fatalf("Expected the flush function to not get called") + } + b := NewWriteBuffer(conf, collector) + for i := uint32(1); i < 10; i++ { + b.Add(i, float64(i*100)) + } + b.flushIfReady() +} + +func TestAddAndGetOutOfOrderOutOfWindow(t *testing.T) { + testData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 4, Val: 400}, + {Ts: 3, Val: 300}, + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 8, Val: 800}, + {Ts: 7, Val: 700}, + {Ts: 9, Val: 900}, + {Ts: 2, Val: 200}, + } + expectedData := []schema.Point{ + {Ts: 5, Val: 500}, + {Ts: 6, Val: 600}, + {Ts: 7, Val: 700}, + {Ts: 8, Val: 800}, + {Ts: 9, Val: 900}, + } + flushedData := []schema.Point{} + // point 2 should be missing because out of reorder window + expectedFlushedData := []schema.Point{ + {Ts: 1, Val: 100}, + {Ts: 3, Val: 300}, + {Ts: 4, Val: 400}, + } + conf := &conf.WriteBufferConf{ReorderWindow: 5, FlushMin: 1} + collector := func(ts uint32, val float64) { + flushedData = append(flushedData, schema.Point{Ts: ts, Val: val}) + } + testAddAndGet(t, conf, collector, testData, expectedData, true) + if !pointSlicesAreEqual(flushedData, expectedFlushedData) { + t.Fatal(fmt.Sprintf("Flushed data does not match expected flushed data: %+v %+v", flushedData, expectedFlushedData)) + } +} + +func TestFlushSortedData(t *testing.T) { + resultI := 0 + results := make([]schema.Point, 400) + receiver := func(ts uint32, val float64) { + results[resultI] = schema.Point{Ts: ts, Val: val} + resultI++ + } + buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1}, receiver) + for i := 100; i < 1100; i++ { + buf.Add(uint32(i), float64(i)) + } + + buf.flushIfReady() + for i := 0; i < 400; i++ { + if results[i].Ts != uint32(i+100) || results[i].Val != float64(i+100) { + t.Fatalf("Unexpected results %+v", results) + } + } +} + +func TestFlushUnsortedData(t *testing.T) { + resultI := 0 + results := make([]schema.Point, 400) + receiver := func(ts uint32, val float64) { + results[resultI] = schema.Point{Ts: ts, Val: val} + resultI++ + } + buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1}, receiver) + data := make([]schema.Point, 1000) + for i := 0; i < 1000; i++ { + data[i] = schema.Point{Ts: uint32(i + 100), Val: float64(i + 100)} + } + unsortedData := unsort(data, 10) + for i := 0; i < len(data); i++ { + buf.Add(unsortedData[i].Ts, unsortedData[i].Val) + } + buf.flushIfReady() + for i := 0; i < 400; i++ { + if results[i].Ts != uint32(i+100) || results[i].Val != float64(i+100) { + t.Fatalf("Unexpected results %+v", results) + } + } +} + +func BenchmarkAddInOrder(b *testing.B) { + data := make([]schema.Point, b.N) + buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: uint32(b.N), FlushMin: 1}, nil) + b.ResetTimer() + + for i := 0; i < b.N; i++ { + buf.Add(data[i].Ts, data[i].Val) + } +} + +func BenchmarkAddOutOfOrder(b *testing.B) { + data := make([]schema.Point, b.N) + unsortedData := unsort(data, 10) + buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: uint32(b.N), FlushMin: 1}, nil) + b.ResetTimer() + + for i := 0; i < b.N; i++ { + buf.Add(unsortedData[i].Ts, unsortedData[i].Val) + } +} + +func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uint32) { + buf := NewWriteBuffer( + &conf.WriteBufferConf{ReorderWindow: reorderWindow, FlushMin: flushMin}, + func(ts uint32, val float64) {}, + ) + ts := uint32(1) + for ; ts <= datapoints; ts++ { + buf.Add(ts, float64(ts*100)) + } + buf.flushIfReady() + + b.ResetTimer() + + for run := 0; run < b.N; run++ { + ts := uint32(1) + for ; ts <= datapoints; ts++ { + buf.Add(ts, float64(ts*100)) + } + buf.flushIfReady() + } +} + +func BenchmarkAddAndFlush10000(b *testing.B) { + benchmarkAddAndFlushX(b, 10000, 100, 1000) +} + +func BenchmarkAddAndFlush1000(b *testing.B) { + benchmarkAddAndFlushX(b, 1000, 10, 100) +} + +func BenchmarkAddAndFlush100(b *testing.B) { + benchmarkAddAndFlushX(b, 100, 1, 10) +} diff --git a/usage/usage_test.go b/usage/usage_test.go index 090d6476df..1e9309cdb9 100644 --- a/usage/usage_test.go +++ b/usage/usage_test.go @@ -53,11 +53,11 @@ func (f *FakeAggMetric) Add(ts uint32, val float64) { } // we won't use this -func (f *FakeAggMetric) Get(from, to uint32) (uint32, []chunk.Iter) { - return 0, make([]chunk.Iter, 0) +func (f *FakeAggMetric) Get(from, to uint32) mdata.MetricResult { + return mdata.MetricResult{Oldest: 0, Iters: make([]chunk.Iter, 0)} } -func (f *FakeAggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) (uint32, []chunk.Iter) { - return 0, make([]chunk.Iter, 0) +func (f *FakeAggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) mdata.MetricResult { + return mdata.MetricResult{Oldest: 0, Iters: make([]chunk.Iter, 0)} } func idFor(org int, metric, unit, mtype string, tags []string, interval uint32) string { From b71f5cf10d36fe9a94d642402990abe64c812a99 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Wed, 12 Jul 2017 02:04:27 -0700 Subject: [PATCH 02/17] reimplement write buffer backed by a slice --- conf/aggregations.go | 12 +- mdata/aggmetric.go | 4 +- mdata/aggmetric_test.go | 12 +- mdata/write_buffer.go | 158 +++++++---------------- mdata/write_buffer_test.go | 257 +++++++++++++++++++++---------------- 5 files changed, 201 insertions(+), 242 deletions(-) diff --git a/conf/aggregations.go b/conf/aggregations.go index 954527fb2d..ea39538bb2 100644 --- a/conf/aggregations.go +++ b/conf/aggregations.go @@ -9,11 +9,6 @@ import ( "github.com/alyu/configparser" ) -type WriteBufferConf struct { - ReorderWindow uint32 - FlushMin uint32 -} - // Aggregations holds the aggregation definitions type Aggregations struct { Data []Aggregation @@ -25,7 +20,7 @@ type Aggregation struct { Pattern *regexp.Regexp XFilesFactor float64 AggregationMethod []Method - WriteBufferConf *WriteBufferConf + ReorderWindow uint32 } // NewAggregations create instance of Aggregations @@ -116,10 +111,7 @@ func ReadAggregations(file string) (Aggregations, error) { } // if reorderWindow == 0 we just disable the buffer if reorderWindow > 0 { - item.WriteBufferConf = &WriteBufferConf{ - ReorderWindow: uint32(reorderWindow), - FlushMin: uint32(flushMin), - } + item.ReorderWindow = uint32(reorderWindow) } } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index c7d717c587..093781f05a 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -64,8 +64,8 @@ func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retent // garbage collected right after creating it, before we can push to it. lastWrite: uint32(time.Now().Unix()), } - if agg != nil && agg.WriteBufferConf != nil { - m.wb = NewWriteBuffer(agg.WriteBufferConf, m.add) + if agg != nil && agg.ReorderWindow != 0 { + m.wb = NewWriteBuffer(agg.ReorderWindow, ret.SecondsPerPoint, m.add) } for _, ret := range retentions[1:] { diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index b0adb85120..e1cb4318d2 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -94,6 +94,7 @@ func (c *Checker) Verify(primary bool, from, to, first, last uint32) { c.t.Fatalf("Raw: Values()=(%v,%v), want end of stream\n", point.Ts, point.Val) } if c.points[index].ts != point.Ts || c.points[index].val != point.Val { + fmt.Println(res.Raw) c.t.Fatalf("Raw: Values()=(%v,%v), want (%v,%v)\n", point.Ts, point.Val, c.points[index].ts, c.points[index].val) } } @@ -239,10 +240,7 @@ func TestAggMetricWithWriteBuffer(t *testing.T) { Pattern: regexp.MustCompile(".*"), XFilesFactor: 0.5, AggregationMethod: []conf.Method{conf.Avg}, - WriteBufferConf: &conf.WriteBufferConf{ - ReorderWindow: 10, - FlushMin: 10, - }, + ReorderWindow: 10, } ret := []conf.Retention{conf.NewRetentionMT(1, 1, 100, 5, true)} c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, &agg, false)) @@ -267,17 +265,17 @@ func TestAggMetricWithWriteBuffer(t *testing.T) { // adds 14 entries that are out of order and the write buffer should order the first 13 // including the previous 7 it will then reach 20 which is = reorder window + flush min, so it causes a flush // the last item (14th) will be added out of order, after the buffer is flushed, so it increases metricsTooOld - for i := uint32(314); i > 300; i-- { + for i := uint32(314); i > 304; i-- { c.Add(i, float64(i)) } - c.DropPointByTs(301) + c.DropPointByTs(305) // get subranges c.Verify(true, 100, 320, 101, 315) // one point has been added out of order and too old for the buffer to reorder if metricsTooOld.Peek() != 1 { - t.Fatalf("Expected the out off order count to be 1") + t.Fatalf("Expected the out of order count to be 1, not %d", metricsTooOld.Peek()) } } diff --git a/mdata/write_buffer.go b/mdata/write_buffer.go index 226339a1f9..fecfd36680 100644 --- a/mdata/write_buffer.go +++ b/mdata/write_buffer.go @@ -1,154 +1,88 @@ package mdata import ( - "fmt" - "sync" - - "github.com/raintank/metrictank/conf" "gopkg.in/raintank/schema.v1" ) -var bufPool = sync.Pool{New: func() interface{} { return &entry{} }} - /* * The write buffer keeps a window of data during which it is ok to send data out of order. - * Once the reorder window plus flush minimum has passed it will try to flush the data out. + * Once the reorder window has passed it will try to flush the data out. * The write buffer itself is not thread safe because it is only used by AggMetric, * which is thread safe, so there is no locking in the buffer. */ type WriteBuffer struct { - reorderWindow uint32 // window size in datapoints during which out of order is allowed - len uint32 // number of datapoints in the buffer - lastFlush uint32 // the timestamp of the last point that's been flushed - flushMin uint32 // min number of datapoints to flush at once - first *entry // first buffer entry - last *entry // last buffer entry - flush func(uint32, float64) // flushing callback + len uint32 // length of buffer in number of datapoints + newest uint32 // index of newest buffer entry + interval uint32 // metric interval + buf []entry // the actual buffer holding the data + flush func(uint32, float64) // flushCount callback } type entry struct { - ts uint32 - val float64 - next, prev *entry + ts uint32 + val float64 } -func NewWriteBuffer(conf *conf.WriteBufferConf, flush func(uint32, float64)) *WriteBuffer { - return &WriteBuffer{ - reorderWindow: conf.ReorderWindow, - flushMin: conf.FlushMin, - flush: flush, +func NewWriteBuffer(reorderWindow uint32, interval int, flush func(uint32, float64)) *WriteBuffer { + buf := &WriteBuffer{ + len: reorderWindow, + flush: flush, + interval: uint32(interval), + newest: 0, + buf: make([]entry, reorderWindow), } + return buf } func (wb *WriteBuffer) Add(ts uint32, val float64) bool { + ts = aggBoundary(ts, wb.interval) + // out of order and too old - if ts < wb.lastFlush { + if wb.buf[wb.newest].ts != 0 && ts <= wb.buf[wb.newest].ts-(wb.len*wb.interval) { return false } - e := bufPool.Get().(*entry) - e.ts = ts - e.val = val - - // initializing the linked list - if wb.first == nil { - e.next = nil - e.prev = nil - wb.first = e - wb.last = e - wb.len++ - } else { - if ts < wb.last.ts { - metricsReordered.Inc() + oldest := (wb.newest + 1) % wb.len + index := (ts / wb.interval) % wb.len + if ts > wb.buf[wb.newest].ts { + flushCount := (ts - wb.buf[wb.newest].ts) / wb.interval + if flushCount > wb.len { + flushCount = wb.len } - // in the normal case data should be added in order, so this will only iterate once - for i := wb.last; i != nil; i = i.prev { - if ts > i.ts { - if i.next == nil { - wb.last = e - } else { - i.next.prev = e - } - e.next = i.next - e.prev = i - i.next = e - e = nil - wb.len++ - break - } - // overwrite value - if ts == i.ts { - i.val = val - e = nil - break + for i := uint32(0); i < flushCount; i++ { + if wb.buf[oldest].ts != 0 { + wb.flush(wb.buf[oldest].ts, wb.buf[oldest].val) } + wb.buf[oldest].ts = 0 + wb.buf[oldest].val = 0 + oldest = (oldest + 1) % wb.len } - if e != nil { - // unlikely case where the added entry is the oldest one present - e.prev = nil - e.next = wb.first - wb.first.prev = e - wb.first = e - wb.len++ - } + wb.buf[index].ts = ts + wb.buf[index].val = val + wb.newest = index + } else { + wb.buf[index].ts = ts + wb.buf[index].val = val } - wb.flushIfReady() - return true } -// if buffer is ready for flushing, this will flush it -func (wb *WriteBuffer) flushIfReady() { - // not enough data, not ready to flush - if wb.len < wb.flushMin+wb.reorderWindow { - return - } - - // we want to flush until the length is equal to the reorder window - flushCount := wb.len - wb.reorderWindow - - nextEntry := wb.first - for i := uint32(0); i < flushCount; i++ { - flushEntry := nextEntry - nextEntry = flushEntry.next - wb.flush(flushEntry.ts, flushEntry.val) - bufPool.Put(flushEntry) - } - - wb.len = wb.reorderWindow - wb.first = nextEntry - wb.first.prev = nil - wb.lastFlush = wb.first.ts -} - -// returns a formatted string that shows the current buffer content, -// only used for debugging purposes and should never be called in prod -func (wb *WriteBuffer) formatted() string { - var str string - var id int - str = fmt.Sprintf("Buffer len: %d first: %p last: %p \n", wb.len, wb.first, wb.last) - for i := wb.first; i != nil; i = i.next { - str = fmt.Sprintf( - "%sId: %d ts: %d val: %f addr: %p prev: %p next: %p\n", - str, id, i.ts, i.val, i, i.prev, i.next, - ) - id++ - } - return str -} - // returns all the data in the buffer as a raw list of points func (wb *WriteBuffer) Get() []schema.Point { res := make([]schema.Point, 0, wb.len) - if wb.first == nil { - return res - } + oldest := (wb.newest + 1) % wb.len - for i := wb.first; i != nil; i = i.next { - res = append(res, schema.Point{Val: i.val, Ts: i.ts}) + for { + if wb.buf[oldest].ts != 0 { + res = append(res, schema.Point{Val: wb.buf[oldest].val, Ts: wb.buf[oldest].ts}) + } + if oldest == wb.newest { + break + } + oldest = (oldest + 1) % wb.len } return res diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index 9d085bfe03..06dac947d4 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -4,19 +4,17 @@ import ( "fmt" "testing" - "github.com/raintank/metrictank/conf" "gopkg.in/raintank/schema.v1" ) -func testAddAndGet(t *testing.T, conf *conf.WriteBufferConf, collector func(uint32, float64), testData, expectedData []schema.Point, expectAddFail bool) { - b := NewWriteBuffer(conf, collector) +func testAddAndGet(t *testing.T, reorderWindow uint32, flush func(uint32, float64), testData, expectedData []schema.Point, expectAddFail bool) { + b := NewWriteBuffer(reorderWindow, 1, flush) gotFailure := false for _, point := range testData { success := b.Add(point.Ts, point.Val) if !success { gotFailure = true } - b.flushIfReady() } if expectAddFail && !gotFailure { t.Fatal("Expected an add to fail, but they all succeeded") @@ -24,10 +22,10 @@ func testAddAndGet(t *testing.T, conf *conf.WriteBufferConf, collector func(uint returned := b.Get() if len(expectedData) != len(returned) { - t.Fatal("Length of returned and testData data unequal") + t.Fatal("Length of returned and testData data unequal", len(returned), len(expectedData)) } if !pointSlicesAreEqual(expectedData, returned) { - t.Fatal(fmt.Sprintf("Returned data does not match testData data %+v, %+v", testData, returned)) + t.Fatal(fmt.Sprintf("Returned data does not match expected data\n%+v\n %+v", testData, expectedData)) } } @@ -57,7 +55,7 @@ func unsort(data []schema.Point, unsortBy int) []schema.Point { return out } -func TestUnsort(t *testing.T) { +func TestWriteBufferUnsort(t *testing.T) { testData := []schema.Point{ {Ts: 0, Val: 0}, {Ts: 1, Val: 100}, @@ -91,176 +89,214 @@ func TestUnsort(t *testing.T) { } } -func TestAddAndGetInOrder(t *testing.T) { +func TestWriteBufferAddAndGetInOrder(t *testing.T) { testData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 2, Val: 200}, - {Ts: 3, Val: 300}, + {Ts: 1001, Val: 100}, + {Ts: 1002, Val: 200}, + {Ts: 1003, Val: 300}, } expectedData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 2, Val: 200}, - {Ts: 3, Val: 300}, + {Ts: 1001, Val: 100}, + {Ts: 1002, Val: 200}, + {Ts: 1003, Val: 300}, } - conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} - testAddAndGet(t, conf, nil, testData, expectedData, false) + flush := func(ts uint32, val float64) {} + testAddAndGet(t, 600, flush, testData, expectedData, false) } -func TestAddAndGetInReverseOrderOutOfWindow(t *testing.T) { +func TestWriteBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { testData := []schema.Point{ - {Ts: 3, Val: 300}, - {Ts: 2, Val: 200}, - {Ts: 1, Val: 100}, + {Ts: 1003, Val: 300}, + {Ts: 1002, Val: 200}, + {Ts: 1001, Val: 100}, } expectedData := []schema.Point{ - {Ts: 3, Val: 300}, + {Ts: 1003, Val: 300}, } - conf := &conf.WriteBufferConf{ReorderWindow: 1, FlushMin: 1} - collector := func(ts uint32, val float64) {} - testAddAndGet(t, conf, collector, testData, expectedData, true) + flush := func(ts uint32, val float64) {} + testAddAndGet(t, 1, flush, testData, expectedData, true) } -func TestAddAndGetOutOfOrderInsideWindow(t *testing.T) { +func TestWriteBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { testData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 2, Val: 200}, - {Ts: 4, Val: 400}, - {Ts: 3, Val: 300}, - {Ts: 5, Val: 500}, - {Ts: 6, Val: 600}, - {Ts: 8, Val: 800}, - {Ts: 7, Val: 700}, - {Ts: 9, Val: 900}, + {Ts: 1001, Val: 100}, + {Ts: 1002, Val: 200}, + {Ts: 1004, Val: 400}, + {Ts: 1003, Val: 300}, + {Ts: 1005, Val: 500}, + {Ts: 1006, Val: 600}, + {Ts: 1008, Val: 800}, + {Ts: 1007, Val: 700}, + {Ts: 1009, Val: 900}, } expectedData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 2, Val: 200}, - {Ts: 3, Val: 300}, - {Ts: 4, Val: 400}, - {Ts: 5, Val: 500}, - {Ts: 6, Val: 600}, - {Ts: 7, Val: 700}, - {Ts: 8, Val: 800}, - {Ts: 9, Val: 900}, - } - conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} - testAddAndGet(t, conf, nil, testData, expectedData, false) + {Ts: 1001, Val: 100}, + {Ts: 1002, Val: 200}, + {Ts: 1003, Val: 300}, + {Ts: 1004, Val: 400}, + {Ts: 1005, Val: 500}, + {Ts: 1006, Val: 600}, + {Ts: 1007, Val: 700}, + {Ts: 1008, Val: 800}, + {Ts: 1009, Val: 900}, + } + flush := func(ts uint32, val float64) {} + testAddAndGet(t, 600, flush, testData, expectedData, false) } -func TestAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { +func TestWriteBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { testData := []schema.Point{ - {Ts: 2, Val: 200}, - {Ts: 4, Val: 400}, - {Ts: 3, Val: 300}, - {Ts: 5, Val: 500}, - {Ts: 1, Val: 100}, - {Ts: 6, Val: 600}, - {Ts: 8, Val: 800}, - {Ts: 7, Val: 700}, - {Ts: 9, Val: 900}, + {Ts: 1002, Val: 200}, + {Ts: 1004, Val: 400}, + {Ts: 1003, Val: 300}, + {Ts: 1005, Val: 500}, + {Ts: 1001, Val: 100}, + {Ts: 1006, Val: 600}, + {Ts: 1008, Val: 800}, + {Ts: 1007, Val: 700}, + {Ts: 1009, Val: 900}, } expectedData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 2, Val: 200}, - {Ts: 3, Val: 300}, - {Ts: 4, Val: 400}, - {Ts: 5, Val: 500}, - {Ts: 6, Val: 600}, - {Ts: 7, Val: 700}, - {Ts: 8, Val: 800}, - {Ts: 9, Val: 900}, - } - conf := &conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1} - testAddAndGet(t, conf, nil, testData, expectedData, false) + {Ts: 1001, Val: 100}, + {Ts: 1002, Val: 200}, + {Ts: 1003, Val: 300}, + {Ts: 1004, Val: 400}, + {Ts: 1005, Val: 500}, + {Ts: 1006, Val: 600}, + {Ts: 1007, Val: 700}, + {Ts: 1008, Val: 800}, + {Ts: 1009, Val: 900}, + } + flush := func(ts uint32, val float64) {} + testAddAndGet(t, 600, flush, testData, expectedData, false) } -func TestOmitFlushIfNotEnoughData(t *testing.T) { - conf := &conf.WriteBufferConf{ReorderWindow: 9, FlushMin: 1} - collector := func(ts uint32, val float64) { +func TestWriteBufferOmitFlushIfNotEnoughData(t *testing.T) { + flush := func(ts uint32, val float64) { t.Fatalf("Expected the flush function to not get called") } - b := NewWriteBuffer(conf, collector) + b := NewWriteBuffer(9, 1, flush) for i := uint32(1); i < 10; i++ { b.Add(i, float64(i*100)) } - b.flushIfReady() } -func TestAddAndGetOutOfOrderOutOfWindow(t *testing.T) { +func TestWriteBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { testData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 4, Val: 400}, - {Ts: 3, Val: 300}, - {Ts: 5, Val: 500}, - {Ts: 6, Val: 600}, - {Ts: 8, Val: 800}, - {Ts: 7, Val: 700}, - {Ts: 9, Val: 900}, - {Ts: 2, Val: 200}, + {Ts: 1001, Val: 100}, + {Ts: 1004, Val: 400}, + {Ts: 1003, Val: 300}, + {Ts: 1005, Val: 500}, + {Ts: 1006, Val: 600}, + {Ts: 1008, Val: 800}, + {Ts: 1007, Val: 700}, + {Ts: 1009, Val: 900}, + {Ts: 1002, Val: 200}, } expectedData := []schema.Point{ - {Ts: 5, Val: 500}, - {Ts: 6, Val: 600}, - {Ts: 7, Val: 700}, - {Ts: 8, Val: 800}, - {Ts: 9, Val: 900}, + {Ts: 1005, Val: 500}, + {Ts: 1006, Val: 600}, + {Ts: 1007, Val: 700}, + {Ts: 1008, Val: 800}, + {Ts: 1009, Val: 900}, } flushedData := []schema.Point{} // point 2 should be missing because out of reorder window expectedFlushedData := []schema.Point{ - {Ts: 1, Val: 100}, - {Ts: 3, Val: 300}, - {Ts: 4, Val: 400}, + {Ts: 1001, Val: 100}, + {Ts: 1003, Val: 300}, + {Ts: 1004, Val: 400}, } - conf := &conf.WriteBufferConf{ReorderWindow: 5, FlushMin: 1} - collector := func(ts uint32, val float64) { + flush := func(ts uint32, val float64) { flushedData = append(flushedData, schema.Point{Ts: ts, Val: val}) } - testAddAndGet(t, conf, collector, testData, expectedData, true) + testAddAndGet(t, 5, flush, testData, expectedData, true) if !pointSlicesAreEqual(flushedData, expectedFlushedData) { - t.Fatal(fmt.Sprintf("Flushed data does not match expected flushed data: %+v %+v", flushedData, expectedFlushedData)) + t.Fatal(fmt.Sprintf("Flushed data does not match expected flushed data:\n%+v\n%+v", flushedData, expectedFlushedData)) } } -func TestFlushSortedData(t *testing.T) { +func TestWriteBufferFlushSortedData(t *testing.T) { resultI := 0 results := make([]schema.Point, 400) receiver := func(ts uint32, val float64) { results[resultI] = schema.Point{Ts: ts, Val: val} resultI++ } - buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1}, receiver) - for i := 100; i < 1100; i++ { - buf.Add(uint32(i), float64(i)) + buf := NewWriteBuffer(600, 1, receiver) + for i := 1100; i < 2100; i++ { + if !buf.Add(uint32(i), float64(i)) { + t.Fatalf("Adding failed") + } } - buf.flushIfReady() for i := 0; i < 400; i++ { - if results[i].Ts != uint32(i+100) || results[i].Val != float64(i+100) { + if results[i].Ts != uint32(i+1100) || results[i].Val != float64(i+1100) { t.Fatalf("Unexpected results %+v", results) } } } -func TestFlushUnsortedData(t *testing.T) { +func TestWriteBufferFlushUnsortedData1(t *testing.T) { + resultI := 0 + results := make([]schema.Point, 5) + receiver := func(ts uint32, val float64) { + results[resultI] = schema.Point{Ts: ts, Val: val} + resultI++ + } + metricsTooOld.SetUint32(0) + buf := NewWriteBuffer(3, 1, receiver) + data := []schema.Point{ + {10, 10}, + {11, 11}, + {9, 9}, + {12, 12}, + {13, 13}, + {20, 20}, + {11, 11}, + {19, 19}, + } + failedCount := 0 + for _, p := range data { + if !buf.Add(p.Ts, p.Val) { + failedCount++ + } + } + expecting := []schema.Point{ + {9, 9}, + {10, 10}, + {11, 11}, + {12, 12}, + {13, 13}, + } + for i, _ := range expecting { + if expecting[i] != results[i] { + t.Fatalf("Unexpected results ", expecting, results) + } + } + if failedCount != 1 { + t.Fatalf("expecting failed count to be 1, not ", failedCount) + } +} + +func TestWriteBufferFlushUnsortedData2(t *testing.T) { resultI := 0 results := make([]schema.Point, 400) receiver := func(ts uint32, val float64) { results[resultI] = schema.Point{Ts: ts, Val: val} resultI++ } - buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: 600, FlushMin: 1}, receiver) + buf := NewWriteBuffer(600, 1, receiver) data := make([]schema.Point, 1000) for i := 0; i < 1000; i++ { - data[i] = schema.Point{Ts: uint32(i + 100), Val: float64(i + 100)} + data[i] = schema.Point{Ts: uint32(i + 1000), Val: float64(i + 1000)} } unsortedData := unsort(data, 10) for i := 0; i < len(data); i++ { buf.Add(unsortedData[i].Ts, unsortedData[i].Val) } - buf.flushIfReady() for i := 0; i < 400; i++ { - if results[i].Ts != uint32(i+100) || results[i].Val != float64(i+100) { + if results[i].Ts != uint32(i+1000) || results[i].Val != float64(i+1000) { t.Fatalf("Unexpected results %+v", results) } } @@ -268,7 +304,7 @@ func TestFlushUnsortedData(t *testing.T) { func BenchmarkAddInOrder(b *testing.B) { data := make([]schema.Point, b.N) - buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: uint32(b.N), FlushMin: 1}, nil) + buf := NewWriteBuffer(uint32(b.N), 1, nil) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -279,7 +315,7 @@ func BenchmarkAddInOrder(b *testing.B) { func BenchmarkAddOutOfOrder(b *testing.B) { data := make([]schema.Point, b.N) unsortedData := unsort(data, 10) - buf := NewWriteBuffer(&conf.WriteBufferConf{ReorderWindow: uint32(b.N), FlushMin: 1}, nil) + buf := NewWriteBuffer(uint32(b.N), 1, nil) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -289,14 +325,14 @@ func BenchmarkAddOutOfOrder(b *testing.B) { func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uint32) { buf := NewWriteBuffer( - &conf.WriteBufferConf{ReorderWindow: reorderWindow, FlushMin: flushMin}, + reorderWindow, + 1, func(ts uint32, val float64) {}, ) ts := uint32(1) for ; ts <= datapoints; ts++ { buf.Add(ts, float64(ts*100)) } - buf.flushIfReady() b.ResetTimer() @@ -305,7 +341,6 @@ func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uin for ; ts <= datapoints; ts++ { buf.Add(ts, float64(ts*100)) } - buf.flushIfReady() } } From 75d20b5c76832dad7f0f20b7590176e6ebb9063b Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Wed, 12 Jul 2017 02:28:32 -0700 Subject: [PATCH 03/17] fixing minor stuff based on PR comments --- api/dataprocessor.go | 2 -- mdata/aggmetric.go | 1 + mdata/write_buffer_test.go | 6 +++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 7117423de0..5d516dc13c 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -370,8 +370,6 @@ func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.MetricResult { if !ok { return mdata.MetricResult{ Oldest: ctx.Req.To, - Iters: make([]chunk.Iter, 0), - Raw: make([]schema.Point, 0), } } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 093781f05a..dcfdbeceed 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -429,6 +429,7 @@ func (a *AggMetric) persist(pos int) { return } +// don't ever call with a ts of 0, cause we use 0 to mean not initialized! func (a *AggMetric) Add(ts uint32, val float64) { a.Lock() defer a.Unlock() diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index 06dac947d4..dbebd0ebda 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -269,13 +269,13 @@ func TestWriteBufferFlushUnsortedData1(t *testing.T) { {12, 12}, {13, 13}, } - for i, _ := range expecting { + for i := range expecting { if expecting[i] != results[i] { - t.Fatalf("Unexpected results ", expecting, results) + t.Fatalf("Unexpected results %+v, %+v", expecting, results) } } if failedCount != 1 { - t.Fatalf("expecting failed count to be 1, not ", failedCount) + t.Fatalf("expecting failed count to be 1, not %d", failedCount) } } From 80b49b345d6b28e8baede466d69dceaef9513a71 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Wed, 12 Jul 2017 10:44:43 -0700 Subject: [PATCH 04/17] fixing more stuff according to comments on PR --- mdata/aggmetric.go | 4 ++-- mdata/aggmetric_test.go | 16 +++++--------- mdata/init.go | 4 ++-- mdata/write_buffer.go | 45 +++++++++++++++++--------------------- mdata/write_buffer_test.go | 16 +++++++------- 5 files changed, 38 insertions(+), 47 deletions(-) diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index dcfdbeceed..0925e03658 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -28,7 +28,7 @@ type AggMetric struct { cachePusher cache.CachePusher sync.RWMutex Key string - wb *WriteBuffer + wb *ReorderBuffer CurrentChunkPos int // element in []Chunks that is active. All others are either finished or nil. NumChunks uint32 // max size of the circular buffer ChunkSpan uint32 // span of individual chunks in seconds @@ -65,7 +65,7 @@ func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retent lastWrite: uint32(time.Now().Unix()), } if agg != nil && agg.ReorderWindow != 0 { - m.wb = NewWriteBuffer(agg.ReorderWindow, ret.SecondsPerPoint, m.add) + m.wb = NewReorderBuffer(agg.ReorderWindow, ret.SecondsPerPoint, m.add) } for _, ret := range retentions[1:] { diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index e1cb4318d2..41f70ebb9f 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -19,11 +19,11 @@ type point struct { val float64 } -type points []point +type ByTs []point -func (a points) Len() int { return len(a) } -func (a points) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a points) Less(i, j int) bool { return a[i].ts < a[j].ts } +func (a ByTs) Len() int { return len(a) } +func (a ByTs) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a ByTs) Less(i, j int) bool { return a[i].ts < a[j].ts } func (p point) String() string { return fmt.Sprintf("point{%0.f at %d}", p.val, p.ts) @@ -46,11 +46,7 @@ func (c *Checker) Add(ts uint32, val float64) { } func (c *Checker) DropPointByTs(ts uint32) { - i := 0 - for { - if i == len(c.points) { - return - } + for i := 0; i != len(c.points); { if c.points[i].ts == ts { c.points = append(c.points[:i], c.points[i+1:]...) } else { @@ -64,7 +60,7 @@ func (c *Checker) DropPointByTs(ts uint32) { // these may be different because AggMetric returns broader rangers (due to packed format), func (c *Checker) Verify(primary bool, from, to, first, last uint32) { currentClusterStatus := cluster.Manager.IsPrimary() - sort.Sort(points(c.points)) + sort.Sort(ByTs(c.points)) cluster.Manager.SetPrimary(primary) res := c.agg.Get(from, to) // we don't do checking or fancy logic, it is assumed that the caller made sure first and last are ts of actual points diff --git a/mdata/init.go b/mdata/init.go index 1a924c3492..4b746a9d9a 100644 --- a/mdata/init.go +++ b/mdata/init.go @@ -26,9 +26,9 @@ var ( // within the reorder window. in such a case they will be inserted in the correct order. // E.g. if the reorder window is 60 (datapoints) then points may be inserted at random order as long as their // ts is not older than the 60th datapoint counting from the newest. - metricsReordered = stats.NewCounter32("tank.metrics_reorderd") + metricsReordered = stats.NewCounter32("tank.metrics_reordered") - // metric tank.metrics_too_old is points that go back in time beyond the scope of the reorder window. + // metric tank.metrics_too_old is points that go back in time beyond the scope of the optional reorder window. // these points will end up being dropped and lost. metricsTooOld = stats.NewCounter32("tank.metrics_too_old") diff --git a/mdata/write_buffer.go b/mdata/write_buffer.go index fecfd36680..9707df05d0 100644 --- a/mdata/write_buffer.go +++ b/mdata/write_buffer.go @@ -11,73 +11,68 @@ import ( * which is thread safe, so there is no locking in the buffer. */ -type WriteBuffer struct { +type ReorderBuffer struct { len uint32 // length of buffer in number of datapoints newest uint32 // index of newest buffer entry interval uint32 // metric interval - buf []entry // the actual buffer holding the data + buf []schema.Point // the actual buffer holding the data flush func(uint32, float64) // flushCount callback } -type entry struct { - ts uint32 - val float64 -} - -func NewWriteBuffer(reorderWindow uint32, interval int, flush func(uint32, float64)) *WriteBuffer { - buf := &WriteBuffer{ +func NewReorderBuffer(reorderWindow uint32, interval int, flush func(uint32, float64)) *ReorderBuffer { + buf := &ReorderBuffer{ len: reorderWindow, flush: flush, interval: uint32(interval), newest: 0, - buf: make([]entry, reorderWindow), + buf: make([]schema.Point, reorderWindow), } return buf } -func (wb *WriteBuffer) Add(ts uint32, val float64) bool { +func (wb *ReorderBuffer) Add(ts uint32, val float64) bool { ts = aggBoundary(ts, wb.interval) // out of order and too old - if wb.buf[wb.newest].ts != 0 && ts <= wb.buf[wb.newest].ts-(wb.len*wb.interval) { + if wb.buf[wb.newest].Ts != 0 && ts <= wb.buf[wb.newest].Ts-(wb.len*wb.interval) { return false } oldest := (wb.newest + 1) % wb.len index := (ts / wb.interval) % wb.len - if ts > wb.buf[wb.newest].ts { - flushCount := (ts - wb.buf[wb.newest].ts) / wb.interval + if ts > wb.buf[wb.newest].Ts { + flushCount := (ts - wb.buf[wb.newest].Ts) / wb.interval if flushCount > wb.len { flushCount = wb.len } for i := uint32(0); i < flushCount; i++ { - if wb.buf[oldest].ts != 0 { - wb.flush(wb.buf[oldest].ts, wb.buf[oldest].val) + if wb.buf[oldest].Ts != 0 { + wb.flush(wb.buf[oldest].Ts, wb.buf[oldest].Val) } - wb.buf[oldest].ts = 0 - wb.buf[oldest].val = 0 + wb.buf[oldest].Ts = 0 + wb.buf[oldest].Val = 0 oldest = (oldest + 1) % wb.len } - wb.buf[index].ts = ts - wb.buf[index].val = val + wb.buf[index].Ts = ts + wb.buf[index].Val = val wb.newest = index } else { - wb.buf[index].ts = ts - wb.buf[index].val = val + wb.buf[index].Ts = ts + wb.buf[index].Val = val } return true } // returns all the data in the buffer as a raw list of points -func (wb *WriteBuffer) Get() []schema.Point { +func (wb *ReorderBuffer) Get() []schema.Point { res := make([]schema.Point, 0, wb.len) oldest := (wb.newest + 1) % wb.len for { - if wb.buf[oldest].ts != 0 { - res = append(res, schema.Point{Val: wb.buf[oldest].val, Ts: wb.buf[oldest].ts}) + if wb.buf[oldest].Ts != 0 { + res = append(res, wb.buf[oldest]) } if oldest == wb.newest { break diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index dbebd0ebda..4172b299cf 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -8,7 +8,7 @@ import ( ) func testAddAndGet(t *testing.T, reorderWindow uint32, flush func(uint32, float64), testData, expectedData []schema.Point, expectAddFail bool) { - b := NewWriteBuffer(reorderWindow, 1, flush) + b := NewReorderBuffer(reorderWindow, 1, flush) gotFailure := false for _, point := range testData { success := b.Add(point.Ts, point.Val) @@ -175,7 +175,7 @@ func TestWriteBufferOmitFlushIfNotEnoughData(t *testing.T) { flush := func(ts uint32, val float64) { t.Fatalf("Expected the flush function to not get called") } - b := NewWriteBuffer(9, 1, flush) + b := NewReorderBuffer(9, 1, flush) for i := uint32(1); i < 10; i++ { b.Add(i, float64(i*100)) } @@ -223,7 +223,7 @@ func TestWriteBufferFlushSortedData(t *testing.T) { results[resultI] = schema.Point{Ts: ts, Val: val} resultI++ } - buf := NewWriteBuffer(600, 1, receiver) + buf := NewReorderBuffer(600, 1, receiver) for i := 1100; i < 2100; i++ { if !buf.Add(uint32(i), float64(i)) { t.Fatalf("Adding failed") @@ -245,7 +245,7 @@ func TestWriteBufferFlushUnsortedData1(t *testing.T) { resultI++ } metricsTooOld.SetUint32(0) - buf := NewWriteBuffer(3, 1, receiver) + buf := NewReorderBuffer(3, 1, receiver) data := []schema.Point{ {10, 10}, {11, 11}, @@ -286,7 +286,7 @@ func TestWriteBufferFlushUnsortedData2(t *testing.T) { results[resultI] = schema.Point{Ts: ts, Val: val} resultI++ } - buf := NewWriteBuffer(600, 1, receiver) + buf := NewReorderBuffer(600, 1, receiver) data := make([]schema.Point, 1000) for i := 0; i < 1000; i++ { data[i] = schema.Point{Ts: uint32(i + 1000), Val: float64(i + 1000)} @@ -304,7 +304,7 @@ func TestWriteBufferFlushUnsortedData2(t *testing.T) { func BenchmarkAddInOrder(b *testing.B) { data := make([]schema.Point, b.N) - buf := NewWriteBuffer(uint32(b.N), 1, nil) + buf := NewReorderBuffer(uint32(b.N), 1, nil) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -315,7 +315,7 @@ func BenchmarkAddInOrder(b *testing.B) { func BenchmarkAddOutOfOrder(b *testing.B) { data := make([]schema.Point, b.N) unsortedData := unsort(data, 10) - buf := NewWriteBuffer(uint32(b.N), 1, nil) + buf := NewReorderBuffer(uint32(b.N), 1, nil) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -324,7 +324,7 @@ func BenchmarkAddOutOfOrder(b *testing.B) { } func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uint32) { - buf := NewWriteBuffer( + buf := NewReorderBuffer( reorderWindow, 1, func(ts uint32, val float64) {}, From 623a9a12bc92f9088ce2d6fb3d69b1c0a1f5bd48 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Thu, 13 Jul 2017 10:24:00 -0700 Subject: [PATCH 05/17] more fixes according to PR comments --- api/dataprocessor.go | 10 +-- mdata/aggmetric.go | 31 ++++----- mdata/aggmetric_test.go | 20 +++--- mdata/ifaces.go | 12 +--- mdata/result.go | 17 +++++ mdata/write_buffer.go | 77 +++++++++++----------- mdata/write_buffer_test.go | 130 +++++++++++++++---------------------- 7 files changed, 140 insertions(+), 157 deletions(-) create mode 100644 mdata/result.go diff --git a/api/dataprocessor.go b/api/dataprocessor.go index 5d516dc13c..bbfd9b95d6 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -316,11 +316,11 @@ func AggMetricKey(key, archive string, aggSpan uint32) string { func (s *Server) getSeriesFixed(req models.Req, consolidator consolidation.Consolidator) []schema.Point { ctx := newRequestContext(&req, consolidator) res := s.getSeries(ctx) - res.Raw = append(s.itersToPoints(ctx, res.Iters), res.Raw...) - return Fix(res.Raw, req.From, req.To, req.ArchInterval) + res.Points = append(s.itersToPoints(ctx, res.Iters), res.Points...) + return Fix(res.Points, req.From, req.To, req.ArchInterval) } -func (s *Server) getSeries(ctx *requestContext) mdata.MetricResult { +func (s *Server) getSeries(ctx *requestContext) mdata.GetResult { res := s.getSeriesAggMetrics(ctx) log.Debug("oldest from aggmetrics is %d", res.Oldest) @@ -365,10 +365,10 @@ func (s *Server) itersToPoints(ctx *requestContext, iters []chunk.Iter) []schema return points } -func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.MetricResult { +func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.GetResult { metric, ok := s.MemoryStore.Get(ctx.Key) if !ok { - return mdata.MetricResult{ + return mdata.GetResult{ Oldest: ctx.Req.To, } } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 0925e03658..30998991f1 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -12,8 +12,6 @@ import ( "github.com/raintank/metrictank/mdata/cache" "github.com/raintank/metrictank/mdata/chunk" "github.com/raintank/worldping-api/pkg/log" - - "gopkg.in/raintank/schema.v1" ) // AggMetric takes in new values, updates the in-memory data and streams the points to aggregators @@ -28,7 +26,7 @@ type AggMetric struct { cachePusher cache.CachePusher sync.RWMutex Key string - wb *ReorderBuffer + rob *ReorderBuffer CurrentChunkPos int // element in []Chunks that is active. All others are either finished or nil. NumChunks uint32 // max size of the circular buffer ChunkSpan uint32 // span of individual chunks in seconds @@ -65,7 +63,7 @@ func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retent lastWrite: uint32(time.Now().Unix()), } if agg != nil && agg.ReorderWindow != 0 { - m.wb = NewReorderBuffer(agg.ReorderWindow, ret.SecondsPerPoint, m.add) + m.rob = NewReorderBuffer(agg.ReorderWindow, ret.SecondsPerPoint) } for _, ret := range retentions[1:] { @@ -139,7 +137,7 @@ func (a *AggMetric) getChunk(pos int) *chunk.Chunk { return a.Chunks[pos] } -func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) MetricResult { +func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) GetResult { // no lock needed cause aggregators don't change at runtime for _, a := range a.aggregators { if a.span == aggSpan { @@ -168,7 +166,7 @@ func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSp // Get all data between the requested time ranges. From is inclusive, to is exclusive. from <= x < to // more data then what's requested may be included // also returns oldest point we have, so that if your query needs data before it, the caller knows when to query cassandra -func (a *AggMetric) Get(from, to uint32) MetricResult { +func (a *AggMetric) Get(from, to uint32) GetResult { pre := time.Now() if LogLevel < 2 { log.Debug("AM %s Get(): %d - %d (%s - %s) span:%ds", a.Key, from, to, TS(from), TS(to), to-from-1) @@ -179,16 +177,14 @@ func (a *AggMetric) Get(from, to uint32) MetricResult { a.RLock() defer a.RUnlock() - result := MetricResult{ + result := GetResult{ Oldest: math.MaxInt32, - Iters: make([]chunk.Iter, 0), - Raw: make([]schema.Point, 0), } - if a.wb != nil { - result.Raw = a.wb.Get() - if len(result.Raw) > 0 { - result.Oldest = result.Raw[0].Ts + if a.rob != nil { + result.Points = a.rob.Get() + if len(result.Points) > 0 { + result.Oldest = result.Points[0].Ts if result.Oldest <= from { return result } @@ -434,13 +430,18 @@ func (a *AggMetric) Add(ts uint32, val float64) { a.Lock() defer a.Unlock() - if a.wb == nil { + if a.rob == nil { // write directly a.add(ts, val) } else { // write through write buffer, returns false if ts is out of reorder window - if !a.wb.Add(ts, val) { + res := a.rob.Add(ts, val) + if !res.Success { metricsTooOld.Inc() + return + } + for _, p := range res.Flushed { + a.add(p.Ts, p.Val) } } } diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index 41f70ebb9f..831aad29a7 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -39,7 +39,6 @@ func NewChecker(t *testing.T, agg *AggMetric) *Checker { return &Checker{t, agg, make([]point, 0)} } -// always add points in ascending order, never same ts! func (c *Checker) Add(ts uint32, val float64) { c.agg.Add(ts, val) c.points = append(c.points, point{ts, val}) @@ -84,14 +83,14 @@ func (c *Checker) Verify(primary bool, from, to, first, last uint32) { } } } - for _, point := range res.Raw { + for _, point := range res.Points { index++ if index > pj { - c.t.Fatalf("Raw: Values()=(%v,%v), want end of stream\n", point.Ts, point.Val) + c.t.Fatalf("Points: Values()=(%v,%v), want end of stream\n", point.Ts, point.Val) } if c.points[index].ts != point.Ts || c.points[index].val != point.Val { - fmt.Println(res.Raw) - c.t.Fatalf("Raw: Values()=(%v,%v), want (%v,%v)\n", point.Ts, point.Val, c.points[index].ts, c.points[index].val) + fmt.Println(res.Points) + c.t.Fatalf("Points: Values()=(%v,%v), want (%v,%v)\n", point.Ts, point.Val, c.points[index].ts, c.points[index].val) } } if index != pj { @@ -228,7 +227,7 @@ func TestAggMetric(t *testing.T) { // c.Verify(true, 800, 1299, 1299, 1299) } -func TestAggMetricWithWriteBuffer(t *testing.T) { +func TestAggMetricWithReorderBuffer(t *testing.T) { cluster.Init("default", "test", time.Now(), "http", 6060) agg := conf.Aggregation{ @@ -241,7 +240,7 @@ func TestAggMetricWithWriteBuffer(t *testing.T) { ret := []conf.Retention{conf.NewRetentionMT(1, 1, 100, 5, true)} c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, &agg, false)) - // basic case, single range + // basic adds and verifies with test data c.Add(101, 101) c.Verify(true, 100, 200, 101, 101) c.Add(105, 105) @@ -250,17 +249,14 @@ func TestAggMetricWithWriteBuffer(t *testing.T) { c.Add(125, 125) c.Add(135, 135) c.Verify(true, 100, 199, 101, 135) - - // add new ranges, aligned and unaligned c.Add(200, 200) c.Add(315, 315) c.Verify(true, 100, 399, 101, 315) metricsTooOld.SetUint32(0) - // adds 14 entries that are out of order and the write buffer should order the first 13 - // including the previous 7 it will then reach 20 which is = reorder window + flush min, so it causes a flush - // the last item (14th) will be added out of order, after the buffer is flushed, so it increases metricsTooOld + // adds 10 entries that are out of order and the write buffer should order the first 9 + // the last item (305) will be too old, so it increases metricsTooOld counter for i := uint32(314); i > 304; i-- { c.Add(i, float64(i)) } diff --git a/mdata/ifaces.go b/mdata/ifaces.go index 4e8c02d96a..455887a813 100644 --- a/mdata/ifaces.go +++ b/mdata/ifaces.go @@ -2,8 +2,6 @@ package mdata import ( "github.com/raintank/metrictank/consolidation" - "github.com/raintank/metrictank/mdata/chunk" - "gopkg.in/raintank/schema.v1" ) type Metrics interface { @@ -13,12 +11,6 @@ type Metrics interface { type Metric interface { Add(ts uint32, val float64) - Get(from, to uint32) MetricResult - GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) MetricResult -} - -type MetricResult struct { - Raw []schema.Point - Iters []chunk.Iter - Oldest uint32 + Get(from, to uint32) GetResult + GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) GetResult } diff --git a/mdata/result.go b/mdata/result.go new file mode 100644 index 0000000000..703f25dd1b --- /dev/null +++ b/mdata/result.go @@ -0,0 +1,17 @@ +package mdata + +import ( + "github.com/raintank/metrictank/mdata/chunk" + "gopkg.in/raintank/schema.v1" +) + +type GetResult struct { + Points []schema.Point + Iters []chunk.Iter + Oldest uint32 +} + +type AddResult struct { + Success bool // defines whether adding the data point has been successful + Flushed []schema.Point // data that's too old to be kept in the buffer, so it gets passed out +} diff --git a/mdata/write_buffer.go b/mdata/write_buffer.go index 9707df05d0..6a6a35136d 100644 --- a/mdata/write_buffer.go +++ b/mdata/write_buffer.go @@ -5,79 +5,78 @@ import ( ) /* - * The write buffer keeps a window of data during which it is ok to send data out of order. - * Once the reorder window has passed it will try to flush the data out. - * The write buffer itself is not thread safe because it is only used by AggMetric, + * The reorder buffer keeps a window of data during which it is ok to send data out of order. + * Once the reorder window has passed Add() will return the old data and delete it from the buffer. + * The reorder buffer itself is not thread safe because it is only used by AggMetric, * which is thread safe, so there is no locking in the buffer. */ type ReorderBuffer struct { - len uint32 // length of buffer in number of datapoints - newest uint32 // index of newest buffer entry - interval uint32 // metric interval - buf []schema.Point // the actual buffer holding the data - flush func(uint32, float64) // flushCount callback + len uint32 // length of buffer in number of datapoints + newest uint32 // index of newest buffer entry + interval uint32 // metric interval + buf []schema.Point // the actual buffer holding the data } -func NewReorderBuffer(reorderWindow uint32, interval int, flush func(uint32, float64)) *ReorderBuffer { +func NewReorderBuffer(reorderWindow uint32, interval int) *ReorderBuffer { buf := &ReorderBuffer{ len: reorderWindow, - flush: flush, interval: uint32(interval), - newest: 0, buf: make([]schema.Point, reorderWindow), } return buf } -func (wb *ReorderBuffer) Add(ts uint32, val float64) bool { - ts = aggBoundary(ts, wb.interval) +func (rob *ReorderBuffer) Add(ts uint32, val float64) AddResult { + ts = aggBoundary(ts, rob.interval) + res := AddResult{} // out of order and too old - if wb.buf[wb.newest].Ts != 0 && ts <= wb.buf[wb.newest].Ts-(wb.len*wb.interval) { - return false + if rob.buf[rob.newest].Ts != 0 && ts <= rob.buf[rob.newest].Ts-(rob.len*rob.interval) { + return res } - oldest := (wb.newest + 1) % wb.len - index := (ts / wb.interval) % wb.len - if ts > wb.buf[wb.newest].Ts { - flushCount := (ts - wb.buf[wb.newest].Ts) / wb.interval - if flushCount > wb.len { - flushCount = wb.len + oldest := (rob.newest + 1) % rob.len + index := (ts / rob.interval) % rob.len + if ts > rob.buf[rob.newest].Ts { + flushCount := (ts - rob.buf[rob.newest].Ts) / rob.interval + if flushCount > rob.len { + flushCount = rob.len } for i := uint32(0); i < flushCount; i++ { - if wb.buf[oldest].Ts != 0 { - wb.flush(wb.buf[oldest].Ts, wb.buf[oldest].Val) + if rob.buf[oldest].Ts != 0 { + res.Flushed = append(res.Flushed, rob.buf[oldest]) } - wb.buf[oldest].Ts = 0 - wb.buf[oldest].Val = 0 - oldest = (oldest + 1) % wb.len + rob.buf[oldest].Ts = 0 + rob.buf[oldest].Val = 0 + oldest = (oldest + 1) % rob.len } - wb.buf[index].Ts = ts - wb.buf[index].Val = val - wb.newest = index + rob.buf[index].Ts = ts + rob.buf[index].Val = val + rob.newest = index } else { - wb.buf[index].Ts = ts - wb.buf[index].Val = val + rob.buf[index].Ts = ts + rob.buf[index].Val = val } - return true + res.Success = true + return res } // returns all the data in the buffer as a raw list of points -func (wb *ReorderBuffer) Get() []schema.Point { - res := make([]schema.Point, 0, wb.len) - oldest := (wb.newest + 1) % wb.len +func (rob *ReorderBuffer) Get() []schema.Point { + res := make([]schema.Point, 0, rob.len) + oldest := (rob.newest + 1) % rob.len for { - if wb.buf[oldest].Ts != 0 { - res = append(res, wb.buf[oldest]) + if rob.buf[oldest].Ts != 0 { + res = append(res, rob.buf[oldest]) } - if oldest == wb.newest { + if oldest == rob.newest { break } - oldest = (oldest + 1) % wb.len + oldest = (oldest + 1) % rob.len } return res diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index 4172b299cf..7879f1edba 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -2,18 +2,22 @@ package mdata import ( "fmt" + "reflect" "testing" "gopkg.in/raintank/schema.v1" ) -func testAddAndGet(t *testing.T, reorderWindow uint32, flush func(uint32, float64), testData, expectedData []schema.Point, expectAddFail bool) { - b := NewReorderBuffer(reorderWindow, 1, flush) +func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail bool) []schema.Point { + var flushed []schema.Point + b := NewReorderBuffer(reorderWindow, 1) gotFailure := false for _, point := range testData { - success := b.Add(point.Ts, point.Val) - if !success { + res := b.Add(point.Ts, point.Val) + if !res.Success { gotFailure = true + } else { + flushed = append(flushed, res.Flushed...) } } if expectAddFail && !gotFailure { @@ -21,26 +25,18 @@ func testAddAndGet(t *testing.T, reorderWindow uint32, flush func(uint32, float6 } returned := b.Get() - if len(expectedData) != len(returned) { - t.Fatal("Length of returned and testData data unequal", len(returned), len(expectedData)) - } - if !pointSlicesAreEqual(expectedData, returned) { + if !reflect.DeepEqual(expectedData, returned) { t.Fatal(fmt.Sprintf("Returned data does not match expected data\n%+v\n %+v", testData, expectedData)) } + return flushed } -func pointSlicesAreEqual(a, b []schema.Point) bool { - if len(a) != len(b) { - return false - } - for i := range a { - if (a[i].Ts != b[i].Ts) || (a[i].Val != b[i].Val) { - return false - } - } - return true -} - +// mixes up a sorted slice +// it simply reverses the order of bunches of a fixed size that's defined by the unsortBy parameter +// for example if unsortBy is 3, then this: +// [0,1,2,3,4,5,6,7,8,9] +// will be turned into this: +// [2,1,0,5,4,3,8,7,6,9] func unsort(data []schema.Point, unsortBy int) []schema.Point { out := make([]schema.Point, len(data)) i := 0 @@ -55,7 +51,7 @@ func unsort(data []schema.Point, unsortBy int) []schema.Point { return out } -func TestWriteBufferUnsort(t *testing.T) { +func TestReorderBufferUnsort(t *testing.T) { testData := []schema.Point{ {Ts: 0, Val: 0}, {Ts: 1, Val: 100}, @@ -89,7 +85,7 @@ func TestWriteBufferUnsort(t *testing.T) { } } -func TestWriteBufferAddAndGetInOrder(t *testing.T) { +func TestReorderBufferAddAndGetInOrder(t *testing.T) { testData := []schema.Point{ {Ts: 1001, Val: 100}, {Ts: 1002, Val: 200}, @@ -100,11 +96,10 @@ func TestWriteBufferAddAndGetInOrder(t *testing.T) { {Ts: 1002, Val: 200}, {Ts: 1003, Val: 300}, } - flush := func(ts uint32, val float64) {} - testAddAndGet(t, 600, flush, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, false) } -func TestWriteBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { +func TestReorderBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { testData := []schema.Point{ {Ts: 1003, Val: 300}, {Ts: 1002, Val: 200}, @@ -113,11 +108,10 @@ func TestWriteBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { expectedData := []schema.Point{ {Ts: 1003, Val: 300}, } - flush := func(ts uint32, val float64) {} - testAddAndGet(t, 1, flush, testData, expectedData, true) + testAddAndGet(t, 1, testData, expectedData, true) } -func TestWriteBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { +func TestReorderBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { testData := []schema.Point{ {Ts: 1001, Val: 100}, {Ts: 1002, Val: 200}, @@ -140,11 +134,10 @@ func TestWriteBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - flush := func(ts uint32, val float64) {} - testAddAndGet(t, 600, flush, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, false) } -func TestWriteBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { +func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { testData := []schema.Point{ {Ts: 1002, Val: 200}, {Ts: 1004, Val: 400}, @@ -167,21 +160,20 @@ func TestWriteBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - flush := func(ts uint32, val float64) {} - testAddAndGet(t, 600, flush, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, false) } -func TestWriteBufferOmitFlushIfNotEnoughData(t *testing.T) { - flush := func(ts uint32, val float64) { - t.Fatalf("Expected the flush function to not get called") - } - b := NewReorderBuffer(9, 1, flush) +func TestReorderBufferOmitFlushIfNotEnoughData(t *testing.T) { + b := NewReorderBuffer(9, 1) for i := uint32(1); i < 10; i++ { - b.Add(i, float64(i*100)) + res := b.Add(i, float64(i*100)) + if len(res.Flushed) > 0 { + t.Fatalf("Expected no data to get flushed out") + } } } -func TestWriteBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { +func TestReorderBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { testData := []schema.Point{ {Ts: 1001, Val: 100}, {Ts: 1004, Val: 400}, @@ -200,34 +192,27 @@ func TestWriteBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - flushedData := []schema.Point{} // point 2 should be missing because out of reorder window expectedFlushedData := []schema.Point{ {Ts: 1001, Val: 100}, {Ts: 1003, Val: 300}, {Ts: 1004, Val: 400}, } - flush := func(ts uint32, val float64) { - flushedData = append(flushedData, schema.Point{Ts: ts, Val: val}) - } - testAddAndGet(t, 5, flush, testData, expectedData, true) - if !pointSlicesAreEqual(flushedData, expectedFlushedData) { + flushedData := testAddAndGet(t, 5, testData, expectedData, true) + if !reflect.DeepEqual(flushedData, expectedFlushedData) { t.Fatal(fmt.Sprintf("Flushed data does not match expected flushed data:\n%+v\n%+v", flushedData, expectedFlushedData)) } } -func TestWriteBufferFlushSortedData(t *testing.T) { - resultI := 0 - results := make([]schema.Point, 400) - receiver := func(ts uint32, val float64) { - results[resultI] = schema.Point{Ts: ts, Val: val} - resultI++ - } - buf := NewReorderBuffer(600, 1, receiver) +func TestReorderBufferFlushSortedData(t *testing.T) { + buf := NewReorderBuffer(600, 1) + var results []schema.Point for i := 1100; i < 2100; i++ { - if !buf.Add(uint32(i), float64(i)) { + res := buf.Add(uint32(i), float64(i)) + if !res.Success { t.Fatalf("Adding failed") } + results = append(results, res.Flushed...) } for i := 0; i < 400; i++ { @@ -237,15 +222,10 @@ func TestWriteBufferFlushSortedData(t *testing.T) { } } -func TestWriteBufferFlushUnsortedData1(t *testing.T) { - resultI := 0 - results := make([]schema.Point, 5) - receiver := func(ts uint32, val float64) { - results[resultI] = schema.Point{Ts: ts, Val: val} - resultI++ - } +func TestReorderBufferFlushUnsortedData1(t *testing.T) { + buf := NewReorderBuffer(3, 1) + var results []schema.Point metricsTooOld.SetUint32(0) - buf := NewReorderBuffer(3, 1, receiver) data := []schema.Point{ {10, 10}, {11, 11}, @@ -258,8 +238,11 @@ func TestWriteBufferFlushUnsortedData1(t *testing.T) { } failedCount := 0 for _, p := range data { - if !buf.Add(p.Ts, p.Val) { + res := buf.Add(p.Ts, p.Val) + if !res.Success { failedCount++ + } else { + results = append(results, res.Flushed...) } } expecting := []schema.Point{ @@ -279,21 +262,17 @@ func TestWriteBufferFlushUnsortedData1(t *testing.T) { } } -func TestWriteBufferFlushUnsortedData2(t *testing.T) { - resultI := 0 - results := make([]schema.Point, 400) - receiver := func(ts uint32, val float64) { - results[resultI] = schema.Point{Ts: ts, Val: val} - resultI++ - } - buf := NewReorderBuffer(600, 1, receiver) +func TestReorderBufferFlushUnsortedData2(t *testing.T) { + buf := NewReorderBuffer(600, 1) + var results []schema.Point data := make([]schema.Point, 1000) for i := 0; i < 1000; i++ { data[i] = schema.Point{Ts: uint32(i + 1000), Val: float64(i + 1000)} } unsortedData := unsort(data, 10) for i := 0; i < len(data); i++ { - buf.Add(unsortedData[i].Ts, unsortedData[i].Val) + res := buf.Add(unsortedData[i].Ts, unsortedData[i].Val) + results = append(results, res.Flushed...) } for i := 0; i < 400; i++ { if results[i].Ts != uint32(i+1000) || results[i].Val != float64(i+1000) { @@ -304,7 +283,7 @@ func TestWriteBufferFlushUnsortedData2(t *testing.T) { func BenchmarkAddInOrder(b *testing.B) { data := make([]schema.Point, b.N) - buf := NewReorderBuffer(uint32(b.N), 1, nil) + buf := NewReorderBuffer(uint32(b.N), 1) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -315,7 +294,7 @@ func BenchmarkAddInOrder(b *testing.B) { func BenchmarkAddOutOfOrder(b *testing.B) { data := make([]schema.Point, b.N) unsortedData := unsort(data, 10) - buf := NewReorderBuffer(uint32(b.N), 1, nil) + buf := NewReorderBuffer(uint32(b.N), 1) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -327,7 +306,6 @@ func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uin buf := NewReorderBuffer( reorderWindow, 1, - func(ts uint32, val float64) {}, ) ts := uint32(1) for ; ts <= datapoints; ts++ { From 95fa1c2695b8d8d9506d56808bdac7c2115d2fb6 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Thu, 13 Jul 2017 11:02:14 -0700 Subject: [PATCH 06/17] remove unnecessary type AddResult --- api/dataprocessor.go | 6 +++--- mdata/aggmetric.go | 12 ++++------- mdata/ifaces.go | 4 ++-- mdata/result.go | 7 +------ mdata/write_buffer.go | 10 ++++----- mdata/write_buffer_test.go | 42 ++++++++++++++++++-------------------- 6 files changed, 35 insertions(+), 46 deletions(-) diff --git a/api/dataprocessor.go b/api/dataprocessor.go index bbfd9b95d6..9939938141 100644 --- a/api/dataprocessor.go +++ b/api/dataprocessor.go @@ -320,7 +320,7 @@ func (s *Server) getSeriesFixed(req models.Req, consolidator consolidation.Conso return Fix(res.Points, req.From, req.To, req.ArchInterval) } -func (s *Server) getSeries(ctx *requestContext) mdata.GetResult { +func (s *Server) getSeries(ctx *requestContext) mdata.Result { res := s.getSeriesAggMetrics(ctx) log.Debug("oldest from aggmetrics is %d", res.Oldest) @@ -365,10 +365,10 @@ func (s *Server) itersToPoints(ctx *requestContext, iters []chunk.Iter) []schema return points } -func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.GetResult { +func (s *Server) getSeriesAggMetrics(ctx *requestContext) mdata.Result { metric, ok := s.MemoryStore.Get(ctx.Key) if !ok { - return mdata.GetResult{ + return mdata.Result{ Oldest: ctx.Req.To, } } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 30998991f1..65213852f7 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -137,7 +137,7 @@ func (a *AggMetric) getChunk(pos int) *chunk.Chunk { return a.Chunks[pos] } -func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) GetResult { +func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) Result { // no lock needed cause aggregators don't change at runtime for _, a := range a.aggregators { if a.span == aggSpan { @@ -166,7 +166,7 @@ func (a *AggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSp // Get all data between the requested time ranges. From is inclusive, to is exclusive. from <= x < to // more data then what's requested may be included // also returns oldest point we have, so that if your query needs data before it, the caller knows when to query cassandra -func (a *AggMetric) Get(from, to uint32) GetResult { +func (a *AggMetric) Get(from, to uint32) Result { pre := time.Now() if LogLevel < 2 { log.Debug("AM %s Get(): %d - %d (%s - %s) span:%ds", a.Key, from, to, TS(from), TS(to), to-from-1) @@ -177,7 +177,7 @@ func (a *AggMetric) Get(from, to uint32) GetResult { a.RLock() defer a.RUnlock() - result := GetResult{ + result := Result{ Oldest: math.MaxInt32, } @@ -436,11 +436,7 @@ func (a *AggMetric) Add(ts uint32, val float64) { } else { // write through write buffer, returns false if ts is out of reorder window res := a.rob.Add(ts, val) - if !res.Success { - metricsTooOld.Inc() - return - } - for _, p := range res.Flushed { + for _, p := range res { a.add(p.Ts, p.Val) } } diff --git a/mdata/ifaces.go b/mdata/ifaces.go index 455887a813..cc90fe2512 100644 --- a/mdata/ifaces.go +++ b/mdata/ifaces.go @@ -11,6 +11,6 @@ type Metrics interface { type Metric interface { Add(ts uint32, val float64) - Get(from, to uint32) GetResult - GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) GetResult + Get(from, to uint32) Result + GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) Result } diff --git a/mdata/result.go b/mdata/result.go index 703f25dd1b..886d2d92d9 100644 --- a/mdata/result.go +++ b/mdata/result.go @@ -5,13 +5,8 @@ import ( "gopkg.in/raintank/schema.v1" ) -type GetResult struct { +type Result struct { Points []schema.Point Iters []chunk.Iter Oldest uint32 } - -type AddResult struct { - Success bool // defines whether adding the data point has been successful - Flushed []schema.Point // data that's too old to be kept in the buffer, so it gets passed out -} diff --git a/mdata/write_buffer.go b/mdata/write_buffer.go index 6a6a35136d..2322dfa576 100644 --- a/mdata/write_buffer.go +++ b/mdata/write_buffer.go @@ -27,15 +27,16 @@ func NewReorderBuffer(reorderWindow uint32, interval int) *ReorderBuffer { return buf } -func (rob *ReorderBuffer) Add(ts uint32, val float64) AddResult { +func (rob *ReorderBuffer) Add(ts uint32, val float64) []schema.Point { ts = aggBoundary(ts, rob.interval) - res := AddResult{} // out of order and too old if rob.buf[rob.newest].Ts != 0 && ts <= rob.buf[rob.newest].Ts-(rob.len*rob.interval) { - return res + metricsTooOld.Inc() + return nil } + var res []schema.Point oldest := (rob.newest + 1) % rob.len index := (ts / rob.interval) % rob.len if ts > rob.buf[rob.newest].Ts { @@ -46,7 +47,7 @@ func (rob *ReorderBuffer) Add(ts uint32, val float64) AddResult { for i := uint32(0); i < flushCount; i++ { if rob.buf[oldest].Ts != 0 { - res.Flushed = append(res.Flushed, rob.buf[oldest]) + res = append(res, rob.buf[oldest]) } rob.buf[oldest].Ts = 0 rob.buf[oldest].Val = 0 @@ -60,7 +61,6 @@ func (rob *ReorderBuffer) Add(ts uint32, val float64) AddResult { rob.buf[index].Val = val } - res.Success = true return res } diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index 7879f1edba..343cab1fe8 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -11,16 +11,12 @@ import ( func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail bool) []schema.Point { var flushed []schema.Point b := NewReorderBuffer(reorderWindow, 1) - gotFailure := false + metricsTooOld.SetUint32(0) for _, point := range testData { - res := b.Add(point.Ts, point.Val) - if !res.Success { - gotFailure = true - } else { - flushed = append(flushed, res.Flushed...) - } + addRes := b.Add(point.Ts, point.Val) + flushed = append(flushed, addRes...) } - if expectAddFail && !gotFailure { + if expectAddFail && metricsTooOld.Peek() == 0 { t.Fatal("Expected an add to fail, but they all succeeded") } returned := b.Get() @@ -166,8 +162,8 @@ func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) func TestReorderBufferOmitFlushIfNotEnoughData(t *testing.T) { b := NewReorderBuffer(9, 1) for i := uint32(1); i < 10; i++ { - res := b.Add(i, float64(i*100)) - if len(res.Flushed) > 0 { + flushed := b.Add(i, float64(i*100)) + if len(flushed) > 0 { t.Fatalf("Expected no data to get flushed out") } } @@ -205,14 +201,15 @@ func TestReorderBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { } func TestReorderBufferFlushSortedData(t *testing.T) { - buf := NewReorderBuffer(600, 1) var results []schema.Point + buf := NewReorderBuffer(600, 1) + metricsTooOld.SetUint32(0) for i := 1100; i < 2100; i++ { - res := buf.Add(uint32(i), float64(i)) - if !res.Success { + flushed := buf.Add(uint32(i), float64(i)) + if metricsTooOld.Peek() != 0 { t.Fatalf("Adding failed") } - results = append(results, res.Flushed...) + results = append(results, flushed...) } for i := 0; i < 400; i++ { @@ -223,9 +220,8 @@ func TestReorderBufferFlushSortedData(t *testing.T) { } func TestReorderBufferFlushUnsortedData1(t *testing.T) { - buf := NewReorderBuffer(3, 1) var results []schema.Point - metricsTooOld.SetUint32(0) + buf := NewReorderBuffer(3, 1) data := []schema.Point{ {10, 10}, {11, 11}, @@ -237,12 +233,14 @@ func TestReorderBufferFlushUnsortedData1(t *testing.T) { {19, 19}, } failedCount := 0 + metricsTooOld.SetUint32(0) for _, p := range data { - res := buf.Add(p.Ts, p.Val) - if !res.Success { + flushed := buf.Add(p.Ts, p.Val) + if metricsTooOld.Peek() != 0 { failedCount++ + metricsTooOld.SetUint32(0) } else { - results = append(results, res.Flushed...) + results = append(results, flushed...) } } expecting := []schema.Point{ @@ -263,16 +261,16 @@ func TestReorderBufferFlushUnsortedData1(t *testing.T) { } func TestReorderBufferFlushUnsortedData2(t *testing.T) { - buf := NewReorderBuffer(600, 1) var results []schema.Point + buf := NewReorderBuffer(600, 1) data := make([]schema.Point, 1000) for i := 0; i < 1000; i++ { data[i] = schema.Point{Ts: uint32(i + 1000), Val: float64(i + 1000)} } unsortedData := unsort(data, 10) for i := 0; i < len(data); i++ { - res := buf.Add(unsortedData[i].Ts, unsortedData[i].Val) - results = append(results, res.Flushed...) + flushed := buf.Add(unsortedData[i].Ts, unsortedData[i].Val) + results = append(results, flushed...) } for i := 0; i < 400; i++ { if results[i].Ts != uint32(i+1000) || results[i].Val != float64(i+1000) { From d078406d0b410d9371d402aec8dd5baea4129390 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 03:39:03 -0700 Subject: [PATCH 07/17] some more renaming from write buffer to reorder buffer --- conf/aggregations.go | 20 ++++++++++---------- mdata/aggmetric.go | 2 +- mdata/aggmetric_test.go | 2 +- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/conf/aggregations.go b/conf/aggregations.go index ea39538bb2..cffd840643 100644 --- a/conf/aggregations.go +++ b/conf/aggregations.go @@ -86,26 +86,26 @@ func ReadAggregations(file string) (Aggregations, error) { } } - writeBufferStr := s.ValueOf("writeBuffer") - if len(writeBufferStr) > 0 { - writeBufferStrs := strings.Split(writeBufferStr, ",") - if len(writeBufferStrs) != 2 { - err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 parts: %s", item.Name, writeBufferStr) + reorderBufferStr := s.ValueOf("reorderBuffer") + if len(reorderBufferStr) > 0 { + reorderBufferStrs := strings.Split(reorderBufferStr, ",") + if len(reorderBufferStrs) != 2 { + err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 parts: %s", item.Name, reorderBufferStr) return Aggregations{}, err } - reorderWindow, err := strconv.ParseUint(writeBufferStrs[0], 10, 32) + reorderWindow, err := strconv.ParseUint(reorderBufferStrs[0], 10, 32) if err != nil { - err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 numbers: %s", item.Name, writeBufferStr) + err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 numbers: %s", item.Name, reorderBufferStr) return Aggregations{}, err } - flushMin, err := strconv.ParseUint(writeBufferStrs[1], 10, 32) + flushMin, err := strconv.ParseUint(reorderBufferStrs[1], 10, 32) if err != nil { - err = fmt.Errorf("[%s]: Failed to parse write buffer conf, expected 2 numbers: %s", item.Name, writeBufferStr) + err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 numbers: %s", item.Name, reorderBufferStr) return Aggregations{}, err } if flushMin < 1 && reorderWindow > 0 { - err = fmt.Errorf("[%s]: Failed to parse write buffer conf, flush minimum needs to be > 0: %s", item.Name, writeBufferStr) + err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, flush minimum needs to be > 0: %s", item.Name, reorderBufferStr) return Aggregations{}, err } diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 65213852f7..4de2985567 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -434,7 +434,7 @@ func (a *AggMetric) Add(ts uint32, val float64) { // write directly a.add(ts, val) } else { - // write through write buffer, returns false if ts is out of reorder window + // write through reorder buffer, returns false if ts is out of reorder window res := a.rob.Add(ts, val) for _, p := range res { a.add(p.Ts, p.Val) diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index 831aad29a7..188d677b22 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -255,7 +255,7 @@ func TestAggMetricWithReorderBuffer(t *testing.T) { metricsTooOld.SetUint32(0) - // adds 10 entries that are out of order and the write buffer should order the first 9 + // adds 10 entries that are out of order and the reorder buffer should order the first 9 // the last item (305) will be too old, so it increases metricsTooOld counter for i := uint32(314); i > 304; i-- { c.Add(i, float64(i)) From 97738a9d299e47ad0081fb8b2a24218290d2556d Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 03:49:51 -0700 Subject: [PATCH 08/17] update docs --- docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf | 1 + docs/config.md | 2 ++ scripts/config/storage-aggregation.conf | 2 ++ 3 files changed, 5 insertions(+) diff --git a/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf b/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf index 465906149c..f8db5e477e 100644 --- a/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf +++ b/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf @@ -2,3 +2,4 @@ pattern = .* xFilesFactor = 0.5 aggregationMethod = avg,min,max +# reorderBuffer = 20 diff --git a/docs/config.md b/docs/config.md index f3c6deddc8..1bb772702e 100644 --- a/docs/config.md +++ b/docs/config.md @@ -415,6 +415,7 @@ retentions = 1s:35d:10min:7 # * aggregationMethod specifies the functions used to aggregate values for the next retention level. Legal methods are avg/average, sum, min, max, and last. The default is average. # Unlike Graphite, you can specify multiple, as it is often handy to have different summaries available depending on what analysis you need to do. # When using multiple, the first one is used for reading. In the future, we will add capabilities to select the different archives for reading. +# * reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, at a defined metric interval, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. # * the settings configured when metrictank starts are what is applied. So you can enable or disable archives by restarting metrictank. # # see https://github.com/raintank/metrictank/blob/master/docs/consolidation.md for related info. @@ -423,6 +424,7 @@ retentions = 1s:35d:10min:7 pattern = .* xFilesFactor = 0.1 aggregationMethod = avg,min,max +# reorderBuffer = 20 ``` This file is generated by [config-to-doc](https://github.com/raintank/metrictank/blob/master/scripts/config-to-doc.sh) diff --git a/scripts/config/storage-aggregation.conf b/scripts/config/storage-aggregation.conf index f3c0947509..5c66ac2001 100644 --- a/scripts/config/storage-aggregation.conf +++ b/scripts/config/storage-aggregation.conf @@ -7,6 +7,7 @@ # * aggregationMethod specifies the functions used to aggregate values for the next retention level. Legal methods are avg/average, sum, min, max, and last. The default is average. # Unlike Graphite, you can specify multiple, as it is often handy to have different summaries available depending on what analysis you need to do. # When using multiple, the first one is used for reading. In the future, we will add capabilities to select the different archives for reading. +# * reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, at a defined metric interval, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. # * the settings configured when metrictank starts are what is applied. So you can enable or disable archives by restarting metrictank. # # see https://github.com/raintank/metrictank/blob/master/docs/consolidation.md for related info. @@ -15,3 +16,4 @@ pattern = .* xFilesFactor = 0.1 aggregationMethod = avg,min,max +# reorderBuffer = 20 From 34f615276c4233670f23d6ac39aa3714536d37c6 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 03:52:54 -0700 Subject: [PATCH 09/17] fix tests --- usage/usage_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/usage/usage_test.go b/usage/usage_test.go index 1e9309cdb9..2dcb4a7f48 100644 --- a/usage/usage_test.go +++ b/usage/usage_test.go @@ -53,11 +53,11 @@ func (f *FakeAggMetric) Add(ts uint32, val float64) { } // we won't use this -func (f *FakeAggMetric) Get(from, to uint32) mdata.MetricResult { - return mdata.MetricResult{Oldest: 0, Iters: make([]chunk.Iter, 0)} +func (f *FakeAggMetric) Get(from, to uint32) mdata.Result { + return mdata.Result{Oldest: 0, Iters: make([]chunk.Iter, 0)} } -func (f *FakeAggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) mdata.MetricResult { - return mdata.MetricResult{Oldest: 0, Iters: make([]chunk.Iter, 0)} +func (f *FakeAggMetric) GetAggregated(consolidator consolidation.Consolidator, aggSpan, from, to uint32) mdata.Result { + return mdata.Result{Oldest: 0, Iters: make([]chunk.Iter, 0)} } func idFor(org int, metric, unit, mtype string, tags []string, interval uint32) string { From 12ef5a176a1f7bf0d932667582bf48c6162636a2 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 04:26:23 -0700 Subject: [PATCH 10/17] remove flushMin --- conf/aggregations.go | 19 ++----------------- mdata/write_buffer_test.go | 8 ++++---- 2 files changed, 6 insertions(+), 21 deletions(-) diff --git a/conf/aggregations.go b/conf/aggregations.go index cffd840643..0b83b5f962 100644 --- a/conf/aggregations.go +++ b/conf/aggregations.go @@ -88,27 +88,12 @@ func ReadAggregations(file string) (Aggregations, error) { reorderBufferStr := s.ValueOf("reorderBuffer") if len(reorderBufferStr) > 0 { - reorderBufferStrs := strings.Split(reorderBufferStr, ",") - if len(reorderBufferStrs) != 2 { - err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 parts: %s", item.Name, reorderBufferStr) - return Aggregations{}, err - } - - reorderWindow, err := strconv.ParseUint(reorderBufferStrs[0], 10, 32) - if err != nil { - err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 numbers: %s", item.Name, reorderBufferStr) - return Aggregations{}, err - } - flushMin, err := strconv.ParseUint(reorderBufferStrs[1], 10, 32) + reorderWindow, err := strconv.ParseUint(reorderBufferStr, 10, 32) if err != nil { - err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected 2 numbers: %s", item.Name, reorderBufferStr) + err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected a number: %s", item.Name, reorderBufferStr) return Aggregations{}, err } - if flushMin < 1 && reorderWindow > 0 { - err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, flush minimum needs to be > 0: %s", item.Name, reorderBufferStr) - return Aggregations{}, err - } // if reorderWindow == 0 we just disable the buffer if reorderWindow > 0 { item.ReorderWindow = uint32(reorderWindow) diff --git a/mdata/write_buffer_test.go b/mdata/write_buffer_test.go index 343cab1fe8..4abf130a97 100644 --- a/mdata/write_buffer_test.go +++ b/mdata/write_buffer_test.go @@ -300,7 +300,7 @@ func BenchmarkAddOutOfOrder(b *testing.B) { } } -func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uint32) { +func benchmarkAddAndFlushX(b *testing.B, datapoints, reorderWindow uint32) { buf := NewReorderBuffer( reorderWindow, 1, @@ -321,13 +321,13 @@ func benchmarkAddAndFlushX(b *testing.B, datapoints, flushMin, reorderWindow uin } func BenchmarkAddAndFlush10000(b *testing.B) { - benchmarkAddAndFlushX(b, 10000, 100, 1000) + benchmarkAddAndFlushX(b, 10000, 1000) } func BenchmarkAddAndFlush1000(b *testing.B) { - benchmarkAddAndFlushX(b, 1000, 10, 100) + benchmarkAddAndFlushX(b, 1000, 100) } func BenchmarkAddAndFlush100(b *testing.B) { - benchmarkAddAndFlushX(b, 100, 1, 10) + benchmarkAddAndFlushX(b, 100, 10) } From 49ec2d61f64d1859d3aee276b5588bcf0a9f1174 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 05:13:13 -0700 Subject: [PATCH 11/17] rename files to reorder buffer --- mdata/{write_buffer.go => reorder_buffer.go} | 0 mdata/{write_buffer_test.go => reorder_buffer_test.go} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mdata/{write_buffer.go => reorder_buffer.go} (100%) rename mdata/{write_buffer_test.go => reorder_buffer_test.go} (100%) diff --git a/mdata/write_buffer.go b/mdata/reorder_buffer.go similarity index 100% rename from mdata/write_buffer.go rename to mdata/reorder_buffer.go diff --git a/mdata/write_buffer_test.go b/mdata/reorder_buffer_test.go similarity index 100% rename from mdata/write_buffer_test.go rename to mdata/reorder_buffer_test.go From 1e89e77c0148eead3dc87752a192465365ef444a Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 05:28:28 -0700 Subject: [PATCH 12/17] fix comment --- mdata/aggmetric.go | 2 +- mdata/reorder_buffer_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 4de2985567..9619e994f4 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -434,7 +434,7 @@ func (a *AggMetric) Add(ts uint32, val float64) { // write directly a.add(ts, val) } else { - // write through reorder buffer, returns false if ts is out of reorder window + // write through reorder buffer res := a.rob.Add(ts, val) for _, p := range res { a.add(p.Ts, p.Val) diff --git a/mdata/reorder_buffer_test.go b/mdata/reorder_buffer_test.go index 4abf130a97..4c7a90a13b 100644 --- a/mdata/reorder_buffer_test.go +++ b/mdata/reorder_buffer_test.go @@ -27,7 +27,7 @@ func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData [] return flushed } -// mixes up a sorted slice +// mixes up a sorted slice, but only within a certain range // it simply reverses the order of bunches of a fixed size that's defined by the unsortBy parameter // for example if unsortBy is 3, then this: // [0,1,2,3,4,5,6,7,8,9] From 03bae1e37f61cd164ca27726aa7084a9b061f72b Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Fri, 14 Jul 2017 09:17:13 -0700 Subject: [PATCH 13/17] move reorder buffer config to storage-schemas & other minor fixes --- conf/aggregations.go | 15 --------------- conf/schemas.go | 22 ++++++++++++++++++---- docs/config.md | 6 +++--- mdata/aggmetric.go | 6 +++--- mdata/aggmetric_test.go | 10 ++++------ mdata/aggmetrics.go | 3 ++- mdata/aggregator.go | 12 ++++++------ mdata/reorder_buffer.go | 1 - mdata/reorder_buffer_test.go | 21 ++++++++++----------- scripts/config/storage-aggregation.conf | 2 -- scripts/config/storage-schemas.conf | 4 +++- 11 files changed, 49 insertions(+), 53 deletions(-) diff --git a/conf/aggregations.go b/conf/aggregations.go index 0b83b5f962..a6be36eaea 100644 --- a/conf/aggregations.go +++ b/conf/aggregations.go @@ -20,7 +20,6 @@ type Aggregation struct { Pattern *regexp.Regexp XFilesFactor float64 AggregationMethod []Method - ReorderWindow uint32 } // NewAggregations create instance of Aggregations @@ -86,20 +85,6 @@ func ReadAggregations(file string) (Aggregations, error) { } } - reorderBufferStr := s.ValueOf("reorderBuffer") - if len(reorderBufferStr) > 0 { - reorderWindow, err := strconv.ParseUint(reorderBufferStr, 10, 32) - if err != nil { - err = fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected a number: %s", item.Name, reorderBufferStr) - return Aggregations{}, err - } - - // if reorderWindow == 0 we just disable the buffer - if reorderWindow > 0 { - item.ReorderWindow = uint32(reorderWindow) - } - } - result.Data = append(result.Data, item) } diff --git a/conf/schemas.go b/conf/schemas.go index 6db5022850..b458b2fbad 100644 --- a/conf/schemas.go +++ b/conf/schemas.go @@ -26,10 +26,11 @@ func (s SchemaSlice) Less(i, j int) bool { return s[i].Priority >= s[j].Priority // Schema represents one schema setting type Schema struct { - Name string - Pattern *regexp.Regexp - Retentions Retentions - Priority int64 + Name string + Pattern *regexp.Regexp + Retentions Retentions + Priority int64 + ReorderWindow uint32 } func NewSchemas(schemas []Schema) Schemas { @@ -116,6 +117,19 @@ func ReadSchemas(file string) (Schemas, error) { } schema.Priority = int64(p)<<32 - int64(i) // to sort records with same priority by position in file + reorderBufferStr := sec.ValueOf("reorderBuffer") + if len(reorderBufferStr) > 0 { + reorderWindow, err := strconv.ParseUint(reorderBufferStr, 10, 32) + if err != nil { + return Schemas{}, fmt.Errorf("[%s]: Failed to parse reorder buffer conf, expected a number: %s", schema.Name, reorderBufferStr) + } + + // if reorderWindow == 0 we just disable the buffer + if reorderWindow > 0 { + schema.ReorderWindow = uint32(reorderWindow) + } + } + schemas = append(schemas, schema) } diff --git a/docs/config.md b/docs/config.md index 1bb772702e..ceedb5c686 100644 --- a/docs/config.md +++ b/docs/config.md @@ -358,8 +358,9 @@ enabled = false # will be applied. The configured rollups will be saved by primary nodes and served in responses if they are ready. # (note in particular that if you remove archives here, we will no longer read from them) # * Retentions must be specified in order of increasing interval and retention +# * The reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, based on the raw interval specified in the first defined retention, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. If you are unsure whether you need this, better leave it disabled to not waste memory. # -# A given rule is made up of 3 lines: the name, regex pattern and retentions. +# A given rule is made up of at least 3 lines: the name, regex pattern, retentions and optionally the reorder buffer size. # The retentions line can specify multiple retention definitions. You need one or more, space separated. # # There are 2 formats for a single retention definition: @@ -401,6 +402,7 @@ enabled = false [default] pattern = .* retentions = 1s:35d:10min:7 +# reorderBuffer = 20 ``` # storage-aggregation.conf @@ -415,7 +417,6 @@ retentions = 1s:35d:10min:7 # * aggregationMethod specifies the functions used to aggregate values for the next retention level. Legal methods are avg/average, sum, min, max, and last. The default is average. # Unlike Graphite, you can specify multiple, as it is often handy to have different summaries available depending on what analysis you need to do. # When using multiple, the first one is used for reading. In the future, we will add capabilities to select the different archives for reading. -# * reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, at a defined metric interval, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. # * the settings configured when metrictank starts are what is applied. So you can enable or disable archives by restarting metrictank. # # see https://github.com/raintank/metrictank/blob/master/docs/consolidation.md for related info. @@ -424,7 +425,6 @@ retentions = 1s:35d:10min:7 pattern = .* xFilesFactor = 0.1 aggregationMethod = avg,min,max -# reorderBuffer = 20 ``` This file is generated by [config-to-doc](https://github.com/raintank/metrictank/blob/master/scripts/config-to-doc.sh) diff --git a/mdata/aggmetric.go b/mdata/aggmetric.go index 9619e994f4..7bd826779d 100644 --- a/mdata/aggmetric.go +++ b/mdata/aggmetric.go @@ -44,7 +44,7 @@ type AggMetric struct { // it optionally also creates aggregations with the given settings // the 0th retention is the native archive of this metric. if there's several others, we create aggregators, using agg. // it's the callers responsibility to make sure agg is not nil in that case! -func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retentions conf.Retentions, agg *conf.Aggregation, dropFirstChunk bool) *AggMetric { +func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retentions conf.Retentions, reorderWindow uint32, agg *conf.Aggregation, dropFirstChunk bool) *AggMetric { // note: during parsing of retentions, we assure there's at least 1. ret := retentions[0] @@ -62,8 +62,8 @@ func NewAggMetric(store Store, cachePusher cache.CachePusher, key string, retent // garbage collected right after creating it, before we can push to it. lastWrite: uint32(time.Now().Unix()), } - if agg != nil && agg.ReorderWindow != 0 { - m.rob = NewReorderBuffer(agg.ReorderWindow, ret.SecondsPerPoint) + if reorderWindow != 0 { + m.rob = NewReorderBuffer(reorderWindow, ret.SecondsPerPoint) } for _, ret := range retentions[1:] { diff --git a/mdata/aggmetric_test.go b/mdata/aggmetric_test.go index 188d677b22..5609ecf304 100644 --- a/mdata/aggmetric_test.go +++ b/mdata/aggmetric_test.go @@ -89,7 +89,6 @@ func (c *Checker) Verify(primary bool, from, to, first, last uint32) { c.t.Fatalf("Points: Values()=(%v,%v), want end of stream\n", point.Ts, point.Val) } if c.points[index].ts != point.Ts || c.points[index].val != point.Val { - fmt.Println(res.Points) c.t.Fatalf("Points: Values()=(%v,%v), want (%v,%v)\n", point.Ts, point.Val, c.points[index].ts, c.points[index].val) } } @@ -123,7 +122,7 @@ func testMetricPersistOptionalPrimary(t *testing.T, primary bool) { numChunks, chunkAddCount, chunkSpan := uint32(5), uint32(10), uint32(300) ret := []conf.Retention{conf.NewRetentionMT(1, 1, chunkSpan, numChunks, true)} - agg := NewAggMetric(dnstore, &mockCache, "foo", ret, nil, false) + agg := NewAggMetric(dnstore, &mockCache, "foo", ret, 0, nil, false) for ts := chunkSpan; ts <= chunkSpan*chunkAddCount; ts += chunkSpan { agg.Add(ts, 1) @@ -159,7 +158,7 @@ func TestAggMetric(t *testing.T) { cluster.Init("default", "test", time.Now(), "http", 6060) ret := []conf.Retention{conf.NewRetentionMT(1, 1, 100, 5, true)} - c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, nil, false)) + c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, 0, nil, false)) // basic case, single range c.Add(101, 101) @@ -235,10 +234,9 @@ func TestAggMetricWithReorderBuffer(t *testing.T) { Pattern: regexp.MustCompile(".*"), XFilesFactor: 0.5, AggregationMethod: []conf.Method{conf.Avg}, - ReorderWindow: 10, } ret := []conf.Retention{conf.NewRetentionMT(1, 1, 100, 5, true)} - c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, &agg, false)) + c := NewChecker(t, NewAggMetric(dnstore, &cache.MockCache{}, "foo", ret, 10, &agg, false)) // basic adds and verifies with test data c.Add(101, 101) @@ -278,7 +276,7 @@ func TestAggMetricDropFirstChunk(t *testing.T) { chunkSpan := uint32(10) numChunks := uint32(5) ret := []conf.Retention{conf.NewRetentionMT(1, 1, chunkSpan, numChunks, true)} - m := NewAggMetric(store, &cache.MockCache{}, "foo", ret, nil, true) + m := NewAggMetric(store, &cache.MockCache{}, "foo", ret, 0, nil, true) m.Add(10, 10) m.Add(11, 11) m.Add(12, 12) diff --git a/mdata/aggmetrics.go b/mdata/aggmetrics.go index a21eae6c53..55ccf7f14b 100644 --- a/mdata/aggmetrics.go +++ b/mdata/aggmetrics.go @@ -86,7 +86,8 @@ func (ms *AggMetrics) GetOrCreate(key, name string, schemaId, aggId uint16) Metr m, ok := ms.Metrics[key] if !ok { agg := Aggregations.Get(aggId) - m = NewAggMetric(ms.store, ms.cachePusher, key, Schemas.Get(schemaId).Retentions, &agg, ms.dropFirstChunk) + schema := Schemas.Get(schemaId) + m = NewAggMetric(ms.store, ms.cachePusher, key, schema.Retentions, schema.ReorderWindow, &agg, ms.dropFirstChunk) ms.Metrics[key] = m metricsActive.Set(len(ms.Metrics)) } diff --git a/mdata/aggregator.go b/mdata/aggregator.go index 8714c85656..39dc620199 100644 --- a/mdata/aggregator.go +++ b/mdata/aggregator.go @@ -45,26 +45,26 @@ func NewAggregator(store Store, cachePusher cache.CachePusher, key string, ret c switch agg { case conf.Avg: if aggregator.sumMetric == nil { - aggregator.sumMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_sum_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.sumMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_sum_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } if aggregator.cntMetric == nil { - aggregator.cntMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_cnt_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.cntMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_cnt_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } case conf.Sum: if aggregator.sumMetric == nil { - aggregator.sumMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_sum_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.sumMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_sum_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } case conf.Lst: if aggregator.lstMetric == nil { - aggregator.lstMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_lst_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.lstMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_lst_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } case conf.Max: if aggregator.maxMetric == nil { - aggregator.maxMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_max_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.maxMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_max_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } case conf.Min: if aggregator.minMetric == nil { - aggregator.minMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_min_%d", key, span), conf.Retentions{ret}, nil, dropFirstChunk) + aggregator.minMetric = NewAggMetric(store, cachePusher, fmt.Sprintf("%s_min_%d", key, span), conf.Retentions{ret}, 0, nil, dropFirstChunk) } } } diff --git a/mdata/reorder_buffer.go b/mdata/reorder_buffer.go index 2322dfa576..e046115cd6 100644 --- a/mdata/reorder_buffer.go +++ b/mdata/reorder_buffer.go @@ -10,7 +10,6 @@ import ( * The reorder buffer itself is not thread safe because it is only used by AggMetric, * which is thread safe, so there is no locking in the buffer. */ - type ReorderBuffer struct { len uint32 // length of buffer in number of datapoints newest uint32 // index of newest buffer entry diff --git a/mdata/reorder_buffer_test.go b/mdata/reorder_buffer_test.go index 4c7a90a13b..a64e816605 100644 --- a/mdata/reorder_buffer_test.go +++ b/mdata/reorder_buffer_test.go @@ -1,14 +1,13 @@ package mdata import ( - "fmt" "reflect" "testing" "gopkg.in/raintank/schema.v1" ) -func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail bool) []schema.Point { +func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail uint32) []schema.Point { var flushed []schema.Point b := NewReorderBuffer(reorderWindow, 1) metricsTooOld.SetUint32(0) @@ -16,13 +15,13 @@ func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData [] addRes := b.Add(point.Ts, point.Val) flushed = append(flushed, addRes...) } - if expectAddFail && metricsTooOld.Peek() == 0 { - t.Fatal("Expected an add to fail, but they all succeeded") + if expectAddFail != metricsTooOld.Peek() { + t.Fatalf("Expected %d failures, but had %d", expectAddFail, metricsTooOld.Peek()) } returned := b.Get() if !reflect.DeepEqual(expectedData, returned) { - t.Fatal(fmt.Sprintf("Returned data does not match expected data\n%+v\n %+v", testData, expectedData)) + t.Fatalf("Returned data does not match expected data\n%+v\n %+v", testData, expectedData) } return flushed } @@ -92,7 +91,7 @@ func TestReorderBufferAddAndGetInOrder(t *testing.T) { {Ts: 1002, Val: 200}, {Ts: 1003, Val: 300}, } - testAddAndGet(t, 600, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, 0) } func TestReorderBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { @@ -104,7 +103,7 @@ func TestReorderBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { expectedData := []schema.Point{ {Ts: 1003, Val: 300}, } - testAddAndGet(t, 1, testData, expectedData, true) + testAddAndGet(t, 1, testData, expectedData, 2) } func TestReorderBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { @@ -130,7 +129,7 @@ func TestReorderBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - testAddAndGet(t, 600, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, 0) } func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { @@ -156,7 +155,7 @@ func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - testAddAndGet(t, 600, testData, expectedData, false) + testAddAndGet(t, 600, testData, expectedData, 0) } func TestReorderBufferOmitFlushIfNotEnoughData(t *testing.T) { @@ -194,9 +193,9 @@ func TestReorderBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { {Ts: 1003, Val: 300}, {Ts: 1004, Val: 400}, } - flushedData := testAddAndGet(t, 5, testData, expectedData, true) + flushedData := testAddAndGet(t, 5, testData, expectedData, 1) if !reflect.DeepEqual(flushedData, expectedFlushedData) { - t.Fatal(fmt.Sprintf("Flushed data does not match expected flushed data:\n%+v\n%+v", flushedData, expectedFlushedData)) + t.Fatalf("Flushed data does not match expected flushed data:\n%+v\n%+v", flushedData, expectedFlushedData) } } diff --git a/scripts/config/storage-aggregation.conf b/scripts/config/storage-aggregation.conf index 5c66ac2001..f3c0947509 100644 --- a/scripts/config/storage-aggregation.conf +++ b/scripts/config/storage-aggregation.conf @@ -7,7 +7,6 @@ # * aggregationMethod specifies the functions used to aggregate values for the next retention level. Legal methods are avg/average, sum, min, max, and last. The default is average. # Unlike Graphite, you can specify multiple, as it is often handy to have different summaries available depending on what analysis you need to do. # When using multiple, the first one is used for reading. In the future, we will add capabilities to select the different archives for reading. -# * reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, at a defined metric interval, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. # * the settings configured when metrictank starts are what is applied. So you can enable or disable archives by restarting metrictank. # # see https://github.com/raintank/metrictank/blob/master/docs/consolidation.md for related info. @@ -16,4 +15,3 @@ pattern = .* xFilesFactor = 0.1 aggregationMethod = avg,min,max -# reorderBuffer = 20 diff --git a/scripts/config/storage-schemas.conf b/scripts/config/storage-schemas.conf index 2228841c62..93fee450ef 100644 --- a/scripts/config/storage-schemas.conf +++ b/scripts/config/storage-schemas.conf @@ -9,8 +9,9 @@ # will be applied. The configured rollups will be saved by primary nodes and served in responses if they are ready. # (note in particular that if you remove archives here, we will no longer read from them) # * Retentions must be specified in order of increasing interval and retention +# * The reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, based on the raw interval specified in the first defined retention, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. If you are unsure whether you need this, better leave it disabled to not waste memory. # -# A given rule is made up of 3 lines: the name, regex pattern and retentions. +# A given rule is made up of at least 3 lines: the name, regex pattern, retentions and optionally the reorder buffer size. # The retentions line can specify multiple retention definitions. You need one or more, space separated. # # There are 2 formats for a single retention definition: @@ -52,3 +53,4 @@ [default] pattern = .* retentions = 1s:35d:10min:7 +# reorderBuffer = 20 From 75c11695932cb5ccaaf40a4e9c7d05330c7e3b2d Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Mon, 17 Jul 2017 00:58:30 -0700 Subject: [PATCH 14/17] update config file comments --- docker/docker-cluster/storage-schemas.conf | 53 +++++++++++++++++++ .../storage-aggregation.conf | 14 ++++- .../storage-schemas.conf | 53 +++++++++++++++++++ 3 files changed, 119 insertions(+), 1 deletion(-) diff --git a/docker/docker-cluster/storage-schemas.conf b/docker/docker-cluster/storage-schemas.conf index d13cde4086..759ee61eb4 100644 --- a/docker/docker-cluster/storage-schemas.conf +++ b/docker/docker-cluster/storage-schemas.conf @@ -1,3 +1,56 @@ +# This config file sets up your retention rules. +# It is an extension of http://graphite.readthedocs.io/en/latest/config-carbon.html#storage-schemas-conf +# Note: +# * You can have 0 to N sections +# * The first match wins, starting from the top. If no match found, we default to single archive of minutely points, retained for 7 days in 2h chunks +# * The patterns are unanchored regular expressions, add '^' or '$' to match the beginning or end of a pattern. +# * When running a cluster of metrictank instances, all instances should have the same agg-settings. +# * Unlike whisper (graphite), the config doesn't stick: if you restart metrictank with updated settings, then those +# will be applied. The configured rollups will be saved by primary nodes and served in responses if they are ready. +# (note in particular that if you remove archives here, we will no longer read from them) +# * Retentions must be specified in order of increasing interval and retention +# * The reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, based on the raw interval specified in the first defined retention, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. If you are unsure whether you need this, better leave it disabled to not waste memory. +# +# A given rule is made up of at least 3 lines: the name, regex pattern, retentions and optionally the reorder buffer size. +# The retentions line can specify multiple retention definitions. You need one or more, space separated. +# +# There are 2 formats for a single retention definition: +# 1) 'series-interval:count-of-datapoints' legacy and not easy to read +# 2) 'series-interval:retention[:chunkspan:numchunks:ready]' more friendly format with optionally 3 extra fields +# +#Series intervals and retentions are specified using the following suffixes: +# +#s - second +#m - minute +#h - hour +#d - day +#y - year +# +# The final 3 fields are specific to metrictank and if unspecified, use sane defaults. +# See https://github.com/raintank/metrictank/blob/master/docs/memory-server.md for more details +# +# chunkspan: duration of chunks. e.g. 10min, 30min, 1h, 90min... +# must be valid value as described here https://github.com/raintank/metrictank/blob/master/docs/memory-server.md#valid-chunk-spans +# Defaults to a the smallest chunkspan that can hold at least 100 points. +# +# numchunks: number of raw chunks to keep in in-memory ring buffer +# See https://github.com/raintank/metrictank/blob/master/docs/memory-server.md for details and trade-offs, especially when compared to chunk-cache +# which may be a more effective method to cache data and alleviate workload for cassandra. +# Defaults to 2 +# +# ready: whether the archive is ready for querying. This is useful if you recently introduced a new archive, but it's still being populated +# so you rather query other archives, even if they don't have the retention to serve your queries +# Defaults to true +# +# Here's an example with multiple retentions: +# [apache_busyWorkers] +# pattern = ^servers\.www.*\.workers\.busyWorkers$ +# retentions = 1s:1d:10min:1,1m:21d,15m:5y:2h:1:false +# +# This example has 3 retention definitions, the first and last override some default options (to use 10minutely and 2hourly chunks and only keep one of them in memory +# and the last rollup is marked as not ready yet for querying. + [default] pattern = .* retentions = 1s:1d +# reorderBuffer = 20 diff --git a/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf b/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf index f8db5e477e..804e096218 100644 --- a/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf +++ b/docker/docker-dev-custom-cfg-kafka/storage-aggregation.conf @@ -1,5 +1,17 @@ +# This config file controls which summaries are created (using which consolidation functions) for your lower-precision archives, as defined in storage-schemas.conf +# It is an extension of http://graphite.readthedocs.io/en/latest/config-carbon.html#storage-aggregation-conf +# Note: +# * This file is optional. If it is not present, we will use avg for everything +# * Anything not matched also uses avg for everything +# * xFilesFactor is not honored yet. What it is in graphite is a floating point number between 0 and 1 specifying what fraction of the previous retention level's slots must have non-null values in order to aggregate to a non-null value. The default is 0.5. +# * aggregationMethod specifies the functions used to aggregate values for the next retention level. Legal methods are avg/average, sum, min, max, and last. The default is average. +# Unlike Graphite, you can specify multiple, as it is often handy to have different summaries available depending on what analysis you need to do. +# When using multiple, the first one is used for reading. In the future, we will add capabilities to select the different archives for reading. +# * the settings configured when metrictank starts are what is applied. So you can enable or disable archives by restarting metrictank. +# +# see https://github.com/raintank/metrictank/blob/master/docs/consolidation.md for related info. + [default] pattern = .* xFilesFactor = 0.5 aggregationMethod = avg,min,max -# reorderBuffer = 20 diff --git a/docker/docker-dev-custom-cfg-kafka/storage-schemas.conf b/docker/docker-dev-custom-cfg-kafka/storage-schemas.conf index 77c2614104..ccd6eb803d 100644 --- a/docker/docker-dev-custom-cfg-kafka/storage-schemas.conf +++ b/docker/docker-dev-custom-cfg-kafka/storage-schemas.conf @@ -1,3 +1,56 @@ +# This config file sets up your retention rules. +# It is an extension of http://graphite.readthedocs.io/en/latest/config-carbon.html#storage-schemas-conf +# Note: +# * You can have 0 to N sections +# * The first match wins, starting from the top. If no match found, we default to single archive of minutely points, retained for 7 days in 2h chunks +# * The patterns are unanchored regular expressions, add '^' or '$' to match the beginning or end of a pattern. +# * When running a cluster of metrictank instances, all instances should have the same agg-settings. +# * Unlike whisper (graphite), the config doesn't stick: if you restart metrictank with updated settings, then those +# will be applied. The configured rollups will be saved by primary nodes and served in responses if they are ready. +# (note in particular that if you remove archives here, we will no longer read from them) +# * Retentions must be specified in order of increasing interval and retention +# * The reorderBuffer an optional buffer that temporarily keeps data points in memory as raw data and allows insertion at random order. The specified value is how many datapoints, based on the raw interval specified in the first defined retention, should be kept before they are flushed out. This is useful if the metric producers cannot guarantee that the data will arrive in order, but it is relatively memory intensive. If you are unsure whether you need this, better leave it disabled to not waste memory. +# +# A given rule is made up of at least 3 lines: the name, regex pattern, retentions and optionally the reorder buffer size. +# The retentions line can specify multiple retention definitions. You need one or more, space separated. +# +# There are 2 formats for a single retention definition: +# 1) 'series-interval:count-of-datapoints' legacy and not easy to read +# 2) 'series-interval:retention[:chunkspan:numchunks:ready]' more friendly format with optionally 3 extra fields +# +#Series intervals and retentions are specified using the following suffixes: +# +#s - second +#m - minute +#h - hour +#d - day +#y - year +# +# The final 3 fields are specific to metrictank and if unspecified, use sane defaults. +# See https://github.com/raintank/metrictank/blob/master/docs/memory-server.md for more details +# +# chunkspan: duration of chunks. e.g. 10min, 30min, 1h, 90min... +# must be valid value as described here https://github.com/raintank/metrictank/blob/master/docs/memory-server.md#valid-chunk-spans +# Defaults to a the smallest chunkspan that can hold at least 100 points. +# +# numchunks: number of raw chunks to keep in in-memory ring buffer +# See https://github.com/raintank/metrictank/blob/master/docs/memory-server.md for details and trade-offs, especially when compared to chunk-cache +# which may be a more effective method to cache data and alleviate workload for cassandra. +# Defaults to 2 +# +# ready: whether the archive is ready for querying. This is useful if you recently introduced a new archive, but it's still being populated +# so you rather query other archives, even if they don't have the retention to serve your queries +# Defaults to true +# +# Here's an example with multiple retentions: +# [apache_busyWorkers] +# pattern = ^servers\.www.*\.workers\.busyWorkers$ +# retentions = 1s:1d:10min:1,1m:21d,15m:5y:2h:1:false +# +# This example has 3 retention definitions, the first and last override some default options (to use 10minutely and 2hourly chunks and only keep one of them in memory +# and the last rollup is marked as not ready yet for querying. + [default] pattern = .* retentions = 1s:35d:2min:2 +# reorderBuffer = 20 From e1ce9ff9df124937fdc6ed5c4a15c197f74094a9 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 17 Jul 2017 14:55:46 +0200 Subject: [PATCH 15/17] fix comment --- mdata/reorder_buffer.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/mdata/reorder_buffer.go b/mdata/reorder_buffer.go index e046115cd6..f190fa63f3 100644 --- a/mdata/reorder_buffer.go +++ b/mdata/reorder_buffer.go @@ -4,12 +4,10 @@ import ( "gopkg.in/raintank/schema.v1" ) -/* - * The reorder buffer keeps a window of data during which it is ok to send data out of order. - * Once the reorder window has passed Add() will return the old data and delete it from the buffer. - * The reorder buffer itself is not thread safe because it is only used by AggMetric, - * which is thread safe, so there is no locking in the buffer. - */ +// ReorderBuffer keeps a window of data during which it is ok to send data out of order. +// Once the reorder window has passed Add() will return the old data and delete it from the buffer. +// The reorder buffer itself is not thread safe because it is only used by AggMetric, +// which is thread safe, so there is no locking in the buffer. type ReorderBuffer struct { len uint32 // length of buffer in number of datapoints newest uint32 // index of newest buffer entry From 4494d554432f1d356e86e6cc9d205f78283b3316 Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Mon, 17 Jul 2017 15:06:59 +0200 Subject: [PATCH 16/17] document metric --- docs/metrics.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/metrics.md b/docs/metrics.md index ac019bf0e7..38e3d30c8b 100644 --- a/docs/metrics.md +++ b/docs/metrics.md @@ -227,6 +227,11 @@ a counter of how many chunks are created the number of times the metrics GC is about to inspect a metric (series) * `tank.metrics_active`: the number of currently known metrics (excl rollup series), measured every second +* `tank.metrics_reordered`: +the number of points received that are going back in time, but are still +within the reorder window. in such a case they will be inserted in the correct order. +E.g. if the reorder window is 60 (datapoints) then points may be inserted at random order as long as their +ts is not older than the 60th datapoint counting from the newest. * `tank.metrics_too_old`: points that go back in time. E.g. for any given series, when a point has a timestamp From 6570db5e21b0266c1b922c7e05a7795e8b294935 Mon Sep 17 00:00:00 2001 From: Mauro Stettler Date: Mon, 17 Jul 2017 06:20:42 -0700 Subject: [PATCH 17/17] use reorder count and test it --- mdata/reorder_buffer.go | 1 + mdata/reorder_buffer_test.go | 16 ++++++++++------ 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/mdata/reorder_buffer.go b/mdata/reorder_buffer.go index f190fa63f3..6bf8c992d2 100644 --- a/mdata/reorder_buffer.go +++ b/mdata/reorder_buffer.go @@ -54,6 +54,7 @@ func (rob *ReorderBuffer) Add(ts uint32, val float64) []schema.Point { rob.buf[index].Val = val rob.newest = index } else { + metricsReordered.Inc() rob.buf[index].Ts = ts rob.buf[index].Val = val } diff --git a/mdata/reorder_buffer_test.go b/mdata/reorder_buffer_test.go index a64e816605..68362ff497 100644 --- a/mdata/reorder_buffer_test.go +++ b/mdata/reorder_buffer_test.go @@ -7,10 +7,11 @@ import ( "gopkg.in/raintank/schema.v1" ) -func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail uint32) []schema.Point { +func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData []schema.Point, expectAddFail, expectReordered uint32) []schema.Point { var flushed []schema.Point b := NewReorderBuffer(reorderWindow, 1) metricsTooOld.SetUint32(0) + metricsReordered.SetUint32(0) for _, point := range testData { addRes := b.Add(point.Ts, point.Val) flushed = append(flushed, addRes...) @@ -18,6 +19,9 @@ func testAddAndGet(t *testing.T, reorderWindow uint32, testData, expectedData [] if expectAddFail != metricsTooOld.Peek() { t.Fatalf("Expected %d failures, but had %d", expectAddFail, metricsTooOld.Peek()) } + if metricsReordered.Peek() != expectReordered { + t.Fatalf("Expected %d metrics to get reordered, but had %d", expectReordered, metricsReordered.Peek()) + } returned := b.Get() if !reflect.DeepEqual(expectedData, returned) { @@ -91,7 +95,7 @@ func TestReorderBufferAddAndGetInOrder(t *testing.T) { {Ts: 1002, Val: 200}, {Ts: 1003, Val: 300}, } - testAddAndGet(t, 600, testData, expectedData, 0) + testAddAndGet(t, 600, testData, expectedData, 0, 0) } func TestReorderBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { @@ -103,7 +107,7 @@ func TestReorderBufferAddAndGetInReverseOrderOutOfWindow(t *testing.T) { expectedData := []schema.Point{ {Ts: 1003, Val: 300}, } - testAddAndGet(t, 1, testData, expectedData, 2) + testAddAndGet(t, 1, testData, expectedData, 2, 0) } func TestReorderBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { @@ -129,7 +133,7 @@ func TestReorderBufferAddAndGetOutOfOrderInsideWindow(t *testing.T) { {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - testAddAndGet(t, 600, testData, expectedData, 0) + testAddAndGet(t, 600, testData, expectedData, 0, 2) } func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) { @@ -155,7 +159,7 @@ func TestReorderBufferAddAndGetOutOfOrderInsideWindowAsFirstPoint(t *testing.T) {Ts: 1008, Val: 800}, {Ts: 1009, Val: 900}, } - testAddAndGet(t, 600, testData, expectedData, 0) + testAddAndGet(t, 600, testData, expectedData, 0, 3) } func TestReorderBufferOmitFlushIfNotEnoughData(t *testing.T) { @@ -193,7 +197,7 @@ func TestReorderBufferAddAndGetOutOfOrderOutOfWindow(t *testing.T) { {Ts: 1003, Val: 300}, {Ts: 1004, Val: 400}, } - flushedData := testAddAndGet(t, 5, testData, expectedData, 1) + flushedData := testAddAndGet(t, 5, testData, expectedData, 1, 2) if !reflect.DeepEqual(flushedData, expectedFlushedData) { t.Fatalf("Flushed data does not match expected flushed data:\n%+v\n%+v", flushedData, expectedFlushedData) }