From e9c29d060db7f6e9a3a6c78b88441fd03b3d988b Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 27 Mar 2024 13:57:10 +0100 Subject: [PATCH 01/24] deltatocumulative: exponential histograms --- processor/deltatocumulativeprocessor/go.mod | 3 + processor/deltatocumulativeprocessor/go.sum | 2 + .../internal/data/add.go | 114 +++++++++++- .../internal/data/exphist_test.go | 167 ++++++++++++++++++ 4 files changed, 283 insertions(+), 3 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/exphist_test.go diff --git a/processor/deltatocumulativeprocessor/go.mod b/processor/deltatocumulativeprocessor/go.mod index cb3fc2f4f9fd..c0e2dd810f1f 100644 --- a/processor/deltatocumulativeprocessor/go.mod +++ b/processor/deltatocumulativeprocessor/go.mod @@ -4,6 +4,7 @@ go 1.21 require ( github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.97.0 + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.97.0 github.com/stretchr/testify v1.9.0 go.opentelemetry.io/collector/component v0.97.0 go.opentelemetry.io/collector/confmap v0.97.0 @@ -57,4 +58,6 @@ require ( replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil => ../../pkg/pdatautil +replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest => ../../pkg/pdatatest + replace github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics => ../../internal/exp/metrics diff --git a/processor/deltatocumulativeprocessor/go.sum b/processor/deltatocumulativeprocessor/go.sum index 16d8df39c0f1..1655950293f9 100644 --- a/processor/deltatocumulativeprocessor/go.sum +++ b/processor/deltatocumulativeprocessor/go.sum @@ -46,6 +46,8 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.97.0 h1:N5b1f66i7KKaUWcDR6gvl5ZucLns2sMidbfXvfQZFTw= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.97.0/go.mod h1:sShrcWdTiwWfuZ8oQhulxYk4eF+qB2vrAxg/MAd690c= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index b40bf05b916d..7783d05ccc7e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -3,7 +3,21 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" -import "go.opentelemetry.io/collector/pdata/pmetric" +import ( + "fmt" + + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +// failure handler +// operations of this package are expected to have no failure cases during +// spec-compliant operation. +// if spec-compliant assumptions are broken however, we want to fail loud +// and clear. can be overwritten during testing +var fail = func(format string, args ...any) { + panic(fmt.Sprintf(format, args...)) +} func (dp Number) Add(in Number) Number { switch in.ValueType() { @@ -23,7 +37,101 @@ func (dp Histogram) Add(in Histogram) Histogram { panic("todo") } -// nolint func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { - panic("todo") + switch { + case dp.Timestamp() >= in.Timestamp(): + fail("out of order") + case dp.Scale() != in.Scale(): + fail("scale changed") + case dp.ZeroCount() != in.ZeroCount(): + fail("zero count changed") + } + + aggregate := func(dpBuckets, inBuckets pmetric.ExponentialHistogramDataPointBuckets) { + var ( + dp = Buckets{data: dpBuckets.BucketCounts(), offset: int(dpBuckets.Offset())} + in = Buckets{data: inBuckets.BucketCounts(), offset: int(inBuckets.Offset())} + aggr = Buckets{data: pcommon.NewUInt64Slice()} + ) + aggr.offset = int(min(dpBuckets.Offset(), inBuckets.Offset())) + if aggr.offset == dp.offset { + aggr.data = dp.data + } + aggr.EnsureLen(max(dp.Len(), in.Len())) + + for i := 0; i < aggr.Len(); i++ { + aggr.SetAt(i, dp.At(i)+in.At(i)) + } + + aggr.CopyTo(dpBuckets) + } + + aggregate(dp.Positive(), in.Positive()) + aggregate(dp.Negative(), in.Negative()) + + count, sum := dp.stats() + dp.SetCount(count) + dp.SetSum(sum) + dp.SetTimestamp(in.Timestamp()) + if dp.HasMin() { + dp.SetMin(min(dp.Min(), in.Min())) + } + if dp.HasMax() { + dp.SetMax(max(dp.Max(), in.Max())) + } + + return dp +} + +func (dp ExpHistogram) stats() (count uint64, sum float64) { + bkt := dp.Positive().BucketCounts() + for i := 0; i < bkt.Len(); i++ { + at := bkt.At(i) + if at != 0 { + count++ + sum += float64(at) + } + } + return count, sum +} + +type Buckets struct { + data pcommon.UInt64Slice + offset int +} + +func (o Buckets) Len() int { + return o.data.Len() + o.offset +} + +func (o Buckets) At(i int) uint64 { + idx, ok := o.idx(i) + if !ok { + return 0 + } + return o.data.At(idx) +} + +func (o Buckets) SetAt(i int, v uint64) { + idx, ok := o.idx(i) + if !ok { + return + } + o.data.SetAt(idx, v) +} + +func (o Buckets) EnsureLen(n int) { + sz := n - o.offset + o.data.EnsureCapacity(sz) + o.data.Append(make([]uint64, sz-o.data.Len())...) +} + +func (o Buckets) idx(i int) (int, bool) { + idx := i - o.offset + return idx, idx >= 0 && idx < o.data.Len() +} + +func (o Buckets) CopyTo(dst pmetric.ExponentialHistogramDataPointBuckets) { + o.data.CopyTo(dst.BucketCounts()) + dst.SetOffset(int32(o.offset)) } diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go new file mode 100644 index 000000000000..c15573265395 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -0,0 +1,167 @@ +package data + +import ( + "testing" + + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +func TestAdd(t *testing.T) { + type tcase struct { + name string + dp, in exphist + want exphist + } + + cases := []tcase{{ + name: "noop", + dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, + in: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, + want: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, + }, { + name: "simple", + dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, + in: exphist{ts: 1, bkt: buckets([]uint64{1, 2, 3, 4, 5, 6, 7, 8, 9})}, + want: exphist{ts: 1, bkt: buckets([]uint64{1, 2, 3, 4, 5, 6, 7, 8, 9})}, + }, { + name: "lower+shorter", + dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 1, 1, 1, 1}, 5)}, + in: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 1, 1, 1, 1, 1}, 2)}, + want: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 1, 1, 1, 2, 2, 1, 1}, 2)}, + }, { + name: "longer", + dp: exphist{ts: 0, bkt: buckets([]uint64{1, 1, 1, 1, 1, 1})}, + in: exphist{ts: 1, bkt: buckets([]uint64{1, 1, 1, 1, 1, 1, 1, 1, 1})}, + want: exphist{ts: 1, bkt: buckets([]uint64{2, 2, 2, 2, 2, 2, 1, 1, 1})}, + }} + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + dp, in := c.dp.Into(), c.in.Into() + want := c.want.Into() + + got := dp.Add(in) + if err := pmetrictest.CompareExponentialHistogramDataPoint(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint); err != nil { + t.Fatal(err) + } + }) + } +} + +func TestBucketsIter(t *testing.T) { + type tcase struct { + name string + data Buckets + want []uint64 + } + + var data = []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9} + cases := []tcase{{ + name: "full", + data: buckets(data, 0, len(data)), + want: []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9}, + }, { + name: "3-6", + data: buckets(data, 3, 6), + want: []uint64{0, 0, 0, 4, 5, 6}, + }} + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + got := make([]uint64, c.data.Len()) + for i := 0; i < c.data.Len(); i++ { + got[i] = c.data.At(i) + } + require.ElementsMatch(t, c.want, got) + require.Equal(t, len(c.want), len(got)) + }) + } +} + +func TestBucketsSet(t *testing.T) { + var ( + ones = []uint64{1, 1, 1, 1, 1, 1, 1, 1, 1} + zero = []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0} + want = []uint64{0, 1, 1, 2, 2, 1, 1, 0, 0} + ) + + var ( + aggr = buckets(zero, 0, len(zero)) + bkt0 = buckets(ones, 1, 5) + bkt1 = buckets(ones, 3, 7) + ) + + for i := 0; i < aggr.Len(); i++ { + aggr.SetAt(i, bkt0.At(i)+bkt1.At(i)) + } + + got := make([]uint64, aggr.Len()) + for i := 0; i < aggr.Len(); i++ { + got[i] = aggr.At(i) + } + require.Equal(t, want, got) +} + +func TestBucketsEnsureLen(t *testing.T) { + var ( + data = []uint64{1, 2, 3, 4} + want = []uint64{0, 0, 3, 4, 0, 0, 0} + ) + bkt := buckets(data, 2, len(data)) + bkt.EnsureLen(len(want)) + + got := make([]uint64, bkt.Len()) + for i := 0; i < bkt.Len(); i++ { + got[i] = bkt.At(i) + } + require.Equal(t, want, got) +} + +func buckets(counts []uint64, bounds ...int) Buckets { + from, to := 0, len(counts) + if len(bounds) > 0 { + from = bounds[0] + } + if len(bounds) > 1 { + to = bounds[1] + } + + data := pcommon.NewUInt64Slice() + data.FromRaw(counts[from:to]) + return Buckets{data: data, offset: from} +} + +type exphist struct { + ts int + bkt Buckets +} + +func (e exphist) Into() ExpHistogram { + dp := pmetric.NewExponentialHistogramDataPoint() + dp.SetTimestamp(pcommon.Timestamp(e.ts)) + + e.bkt.data.CopyTo(dp.Positive().BucketCounts()) + dp.Positive().SetOffset(int32(e.bkt.offset)) + e.bkt.data.CopyTo(dp.Negative().BucketCounts()) + dp.Negative().SetOffset(int32(e.bkt.offset)) + + var ( + count uint64 + sum float64 + ) + pos := dp.Positive().BucketCounts() + for i := 0; i < pos.Len(); i++ { + at := pos.At(i) + if at != 0 { + count++ + sum += float64(pos.At(i)) + } + } + dp.SetCount(count) + dp.SetSum(sum) + + return ExpHistogram{ExponentialHistogramDataPoint: dp} +} From 1f9743d627440381753195676a65edf742d29669 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 10 Apr 2024 19:56:45 +0200 Subject: [PATCH 02/24] exphist: properly handle min,max,sum,count --- .../internal/data/add.go | 42 ++--- .../internal/data/exphist_test.go | 146 ++++++++++-------- .../internal/data/helper_test.go | 117 ++++++++++++++ 3 files changed, 216 insertions(+), 89 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/helper_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 7783d05ccc7e..7fb235384c4b 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -4,8 +4,6 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" import ( - "fmt" - "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" ) @@ -15,8 +13,8 @@ import ( // spec-compliant operation. // if spec-compliant assumptions are broken however, we want to fail loud // and clear. can be overwritten during testing -var fail = func(format string, args ...any) { - panic(fmt.Sprintf(format, args...)) +var fail = func(msg string) { + panic(msg) } func (dp Number) Add(in Number) Number { @@ -69,30 +67,32 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { aggregate(dp.Positive(), in.Positive()) aggregate(dp.Negative(), in.Negative()) - count, sum := dp.stats() - dp.SetCount(count) - dp.SetSum(sum) dp.SetTimestamp(in.Timestamp()) - if dp.HasMin() { - dp.SetMin(min(dp.Min(), in.Min())) + dp.SetCount(dp.Count() + in.Count()) + + type T = ExpHistogram + optionals := []field{ + {get: T.Sum, set: T.SetSum, has: T.HasSum, del: T.RemoveSum, op: func(a, b float64) float64 { return a + b }}, + {get: T.Min, set: T.SetMin, has: T.HasMin, del: T.RemoveMin, op: func(a, b float64) float64 { return min(a, b) }}, + {get: T.Max, set: T.SetMax, has: T.HasMax, del: T.RemoveMax, op: func(a, b float64) float64 { return max(a, b) }}, } - if dp.HasMax() { - dp.SetMax(max(dp.Max(), in.Max())) + for _, f := range optionals { + if f.has(dp) && f.has(in) { + f.set(dp, f.op(f.get(dp), f.get(in))) + } else { + f.del(dp) + } } return dp } -func (dp ExpHistogram) stats() (count uint64, sum float64) { - bkt := dp.Positive().BucketCounts() - for i := 0; i < bkt.Len(); i++ { - at := bkt.At(i) - if at != 0 { - count++ - sum += float64(at) - } - } - return count, sum +type field struct { + get func(ExpHistogram) float64 + set func(ExpHistogram, float64) + has func(ExpHistogram) bool + del func(ExpHistogram) + op func(a, b float64) float64 } type Buckets struct { diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go index c15573265395..3adc46a30488 100644 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -1,6 +1,7 @@ package data import ( + "math" "testing" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" @@ -9,39 +10,57 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" ) +// represents none/absent/unset in several tests +const ø = math.MaxUint64 + func TestAdd(t *testing.T) { type tcase struct { name string - dp, in exphist - want exphist + dp, in expdp + want expdp } cases := []tcase{{ name: "noop", - dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, - in: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, - want: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, + dp: expdp{ts: 0, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + in: expdp{ts: 1, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + want: expdp{ts: 1, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, }, { name: "simple", - dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 0, 0, 0, 0})}, - in: exphist{ts: 1, bkt: buckets([]uint64{1, 2, 3, 4, 5, 6, 7, 8, 9})}, - want: exphist{ts: 1, bkt: buckets([]uint64{1, 2, 3, 4, 5, 6, 7, 8, 9})}, + dp: expdp{ts: 0, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + in: expdp{ts: 1, pos: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * 45}, + want: expdp{ts: 1, pos: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * (0 + 45)}, }, { name: "lower+shorter", - dp: exphist{ts: 0, bkt: buckets([]uint64{0, 0, 0, 0, 0, 1, 1, 1, 1}, 5)}, - in: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 1, 1, 1, 1, 1}, 2)}, - want: exphist{ts: 1, bkt: buckets([]uint64{0, 0, 1, 1, 1, 2, 2, 1, 1}, 2)}, + dp: expdp{ts: 0, pos: buckets{ø, ø, ø, ø, ø, 1, 1, 1, 1}, neg: buckets{ø, ø, ø, ø, ø, 1, 1, 1, 1}, count: 2 * 4}, + in: expdp{ts: 1, pos: buckets{ø, ø, 1, 1, 1, 1, 1, ø, ø}, neg: buckets{ø, ø, 1, 1, 1, 1, 1, ø, ø}, count: 2 * 5}, + want: expdp{ts: 1, pos: buckets{ø, ø, 1, 1, 1, 2, 2, 1, 1}, neg: buckets{ø, ø, 1, 1, 1, 2, 2, 1, 1}, count: 2 * (4 + 5)}, }, { name: "longer", - dp: exphist{ts: 0, bkt: buckets([]uint64{1, 1, 1, 1, 1, 1})}, - in: exphist{ts: 1, bkt: buckets([]uint64{1, 1, 1, 1, 1, 1, 1, 1, 1})}, - want: exphist{ts: 1, bkt: buckets([]uint64{2, 2, 2, 2, 2, 2, 1, 1, 1})}, + dp: expdp{ts: 0, pos: buckets{1, 1, 1, 1, 1, 1, ø, ø, ø}, neg: buckets{1, 1, 1, 1, 1, 1, ø, ø, ø}, count: 2 * 6}, + in: expdp{ts: 1, pos: buckets{1, 1, 1, 1, 1, 1, 1, 1, 1}, neg: buckets{1, 1, 1, 1, 1, 1, 1, 1, 1}, count: 2 * 9}, + want: expdp{ts: 1, pos: buckets{2, 2, 2, 2, 2, 2, 1, 1, 1}, neg: buckets{2, 2, 2, 2, 2, 2, 1, 1, 1}, count: 2 * (6 + 9)}, + }, { + name: "optional/missing-dp", + dp: expdp{ts: 0, pos: zeros(5).observe0(0.6, 2.4) /* */, count: 2}, + in: expdp{ts: 1, pos: zeros(5).observe0(1.5, 3.2, 6.3), min: some(1.5), max: some(6.3), sum: some(11.0), count: 3}, + want: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4, 1.5, 3.2, 6.3) /* */, count: 5}, + }, { + name: "optional/missing-in", + dp: expdp{ts: 0, pos: zeros(5).observe0(1.5, 3.2, 6.3), min: some(1.5), max: some(6.3), sum: some(11.0), count: 3}, + in: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4) /* */, count: 2}, + want: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4, 1.5, 3.2, 6.3) /* */, count: 5}, + }, { + name: "min-max-sum", + dp: expdp{ts: 0, pos: zeros(5).observe0(1.5, 5.3, 11.6) /* */, min: some(1.5), max: some(11.6), sum: some(18.4), count: 3}, + in: expdp{ts: 1, pos: zeros(5).observe0(0.6, 3.3, 7.9) /* */, min: some(0.6), max: some(07.9), sum: some(11.8), count: 3}, + want: expdp{ts: 1, pos: zeros(5).observe0(1.5, 5.3, 11.6, 0.6, 3.3, 7.9), min: some(0.6), max: some(11.6), sum: some(30.2), count: 6}, }} for _, c := range cases { t.Run(c.name, func(t *testing.T) { - dp, in := c.dp.Into(), c.in.Into() - want := c.want.Into() + dp, in := c.dp.into(), c.in.into() + want := c.want.into() got := dp.Add(in) if err := pmetrictest.CompareExponentialHistogramDataPoint(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint); err != nil { @@ -58,15 +77,14 @@ func TestBucketsIter(t *testing.T) { want []uint64 } - var data = []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9} cases := []tcase{{ name: "full", - data: buckets(data, 0, len(data)), - want: []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9}, + data: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}.into(), + want: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, }, { name: "3-6", - data: buckets(data, 3, 6), - want: []uint64{0, 0, 0, 4, 5, 6}, + data: buckets{ø, ø, ø, 4, 5, 6, ø, ø, ø, ø}.into(), + want: buckets{0, 0, 0, 4, 5, 6}, }} for _, c := range cases { @@ -83,22 +101,17 @@ func TestBucketsIter(t *testing.T) { func TestBucketsSet(t *testing.T) { var ( - ones = []uint64{1, 1, 1, 1, 1, 1, 1, 1, 1} - zero = []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0} - want = []uint64{0, 1, 1, 2, 2, 1, 1, 0, 0} - ) - - var ( - aggr = buckets(zero, 0, len(zero)) - bkt0 = buckets(ones, 1, 5) - bkt1 = buckets(ones, 3, 7) + aggr = buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}.into() + bkt0 = buckets{ø, 1, 1, 1, 1, ø, ø, ø, ø}.into() + bkt1 = buckets{ø, ø, ø, 1, 1, 1, 1, ø, ø}.into() + want = buckets{0, 1, 1, 2, 2, 1, 1, 0, 0} ) for i := 0; i < aggr.Len(); i++ { aggr.SetAt(i, bkt0.At(i)+bkt1.At(i)) } - got := make([]uint64, aggr.Len()) + got := make(buckets, aggr.Len()) for i := 0; i < aggr.Len(); i++ { got[i] = aggr.At(i) } @@ -107,61 +120,58 @@ func TestBucketsSet(t *testing.T) { func TestBucketsEnsureLen(t *testing.T) { var ( - data = []uint64{1, 2, 3, 4} - want = []uint64{0, 0, 3, 4, 0, 0, 0} + data = buckets{ø, ø, 3, 4} + want = buckets{0, 0, 3, 4, 0, 0, 0} ) - bkt := buckets(data, 2, len(data)) + bkt := data.into() bkt.EnsureLen(len(want)) - got := make([]uint64, bkt.Len()) + got := make(buckets, bkt.Len()) for i := 0; i < bkt.Len(); i++ { got[i] = bkt.At(i) } require.Equal(t, want, got) } -func buckets(counts []uint64, bounds ...int) Buckets { - from, to := 0, len(counts) - if len(bounds) > 0 { - from = bounds[0] - } - if len(bounds) > 1 { - to = bounds[1] - } +type expdp struct { + ts int + pos buckets + neg buckets - data := pcommon.NewUInt64Slice() - data.FromRaw(counts[from:to]) - return Buckets{data: data, offset: from} -} + scale int32 + count uint64 + sum *float64 -type exphist struct { - ts int - bkt Buckets + min, max *float64 } -func (e exphist) Into() ExpHistogram { +func (e expdp) into() ExpHistogram { dp := pmetric.NewExponentialHistogramDataPoint() dp.SetTimestamp(pcommon.Timestamp(e.ts)) - e.bkt.data.CopyTo(dp.Positive().BucketCounts()) - dp.Positive().SetOffset(int32(e.bkt.offset)) - e.bkt.data.CopyTo(dp.Negative().BucketCounts()) - dp.Negative().SetOffset(int32(e.bkt.offset)) + pos := e.pos.into() + pos.data.CopyTo(dp.Positive().BucketCounts()) + dp.Positive().SetOffset(int32(pos.offset)) - var ( - count uint64 - sum float64 - ) - pos := dp.Positive().BucketCounts() - for i := 0; i < pos.Len(); i++ { - at := pos.At(i) - if at != 0 { - count++ - sum += float64(pos.At(i)) - } + neg := e.neg.into() + neg.data.CopyTo(dp.Negative().BucketCounts()) + dp.Negative().SetOffset(int32(neg.offset)) + + dp.SetScale(e.scale) + dp.SetCount(e.count) + if e.sum != nil { + dp.SetSum(*e.sum) + } + if e.min != nil { + dp.SetMin(*e.min) + } + if e.max != nil { + dp.SetMax(*e.max) } - dp.SetCount(count) - dp.SetSum(sum) return ExpHistogram{ExponentialHistogramDataPoint: dp} } + +func some[T any](v T) *T { + return &v +} diff --git a/processor/deltatocumulativeprocessor/internal/data/helper_test.go b/processor/deltatocumulativeprocessor/internal/data/helper_test.go new file mode 100644 index 000000000000..daa0753d373c --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/helper_test.go @@ -0,0 +1,117 @@ +package data + +import ( + "math" + "sort" + "testing" + + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/pdata/pcommon" +) + +type buckets []uint64 + +func (b buckets) into() Buckets { + start := 0 + for i := 0; i < len(b); i++ { + if b[i] != ø { + start = i + break + } + } + + end := len(b) + for i := start; i < len(b); i++ { + if b[i] == ø { + end = i + break + } + } + + data := pcommon.NewUInt64Slice() + data.FromRaw([]uint64(b[start:end])) + return Buckets{ + data: data, + offset: start, + } +} + +func TestBucketsHelper(t *testing.T) { + uints := func(data ...uint64) pcommon.UInt64Slice { + us := pcommon.NewUInt64Slice() + us.FromRaw(data) + return us + } + + cases := []struct { + bkts buckets + want Buckets + }{{ + bkts: buckets{}, + want: Buckets{data: uints(), offset: 0}, + }, { + bkts: buckets{1, 2, 3, 4}, + want: Buckets{data: uints(1, 2, 3, 4), offset: 0}, + }, { + bkts: buckets{ø, ø, 3, 4}, + want: Buckets{data: uints(3, 4), offset: 2}, + }, { + bkts: buckets{ø, ø, 3, 4, ø, ø}, + want: Buckets{data: uints(3, 4), offset: 2}, + }, { + bkts: buckets{1, 2, ø, ø}, + want: Buckets{data: uints(1, 2), offset: 0}, + }} + + for _, c := range cases { + got := c.bkts.into() + require.Equal(t, c.want, got) + } +} + +func zeros(size int) buckets { + return make(buckets, size) +} + +// observe some points with scale 0 +func (dps buckets) observe0(pts ...float64) buckets { + if len(pts) == 0 { + return dps + } + + idx := func(v float64) int { + return int(math.Ceil(math.Log2(v))) + } + + sort.Float64s(pts) + min := idx(pts[0]) + max := idx(pts[len(pts)-1]) + for i := min; i <= max; i++ { + if dps[i] == ø { + dps[i] = 0 + } + } + + for _, pt := range pts { + dps[idx(pt)] += 1 + } + return dps +} + +func TestObserve0(t *testing.T) { + cases := []struct { + pts []float64 + want buckets + }{{ + pts: []float64{1.5, 5.3, 11.6}, + want: buckets{0, 1, 0, 1, 1}, + }, { + pts: []float64{0.6, 3.3, 7.9}, + want: buckets{1, 0, 1, 1, 0}, + }} + + for _, c := range cases { + got := zeros(len(c.want)).observe0(c.pts...) + require.Equal(t, c.want, got) + } +} From 7311667904c299f9ace84fb45a1fd4fbf27e2455 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 17 Apr 2024 13:34:02 +0200 Subject: [PATCH 03/24] data: expo.Merge introduces expo package for exponential histogram operations, mainly expo.Merge and expo.Scale --- processor/deltatocumulativeprocessor/go.mod | 1 + processor/deltatocumulativeprocessor/go.sum | 2 + .../internal/data/add.go | 90 +++---------- .../internal/data/exphist_test.go | 122 +++++++++++------ .../internal/data/expo/buckets.go | 86 ++++++++++++ .../internal/data/expo/expotest/bins.go | 41 ++++++ .../internal/data/expo/merge.go | 51 ++++++++ .../internal/data/expo/merge_test.go | 52 ++++++++ .../internal/data/expo/scale.go | 18 +++ .../internal/data/expo/zero.go | 27 ++++ .../internal/data/helper_test.go | 123 ++++++++++++------ 11 files changed, 459 insertions(+), 154 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/buckets.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/merge.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/scale.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/zero.go diff --git a/processor/deltatocumulativeprocessor/go.mod b/processor/deltatocumulativeprocessor/go.mod index c0e2dd810f1f..078157b6fbd6 100644 --- a/processor/deltatocumulativeprocessor/go.mod +++ b/processor/deltatocumulativeprocessor/go.mod @@ -3,6 +3,7 @@ module github.com/open-telemetry/opentelemetry-collector-contrib/processor/delta go 1.21 require ( + github.com/matryer/is v1.4.1 github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.97.0 github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.97.0 github.com/stretchr/testify v1.9.0 diff --git a/processor/deltatocumulativeprocessor/go.sum b/processor/deltatocumulativeprocessor/go.sum index 1655950293f9..8b4d3141dd68 100644 --- a/processor/deltatocumulativeprocessor/go.sum +++ b/processor/deltatocumulativeprocessor/go.sum @@ -37,6 +37,8 @@ github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/matryer/is v1.4.1 h1:55ehd8zaGABKLXQUe2awZ99BD/PTc2ls+KV/dXphgEQ= +github.com/matryer/is v1.4.1/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 7fb235384c4b..80be60831690 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -4,19 +4,12 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" import ( - "go.opentelemetry.io/collector/pdata/pcommon" + "math" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" ) -// failure handler -// operations of this package are expected to have no failure cases during -// spec-compliant operation. -// if spec-compliant assumptions are broken however, we want to fail loud -// and clear. can be overwritten during testing -var fail = func(msg string) { - panic(msg) -} - func (dp Number) Add(in Number) Number { switch in.ValueType() { case pmetric.NumberDataPointValueTypeDouble: @@ -38,43 +31,28 @@ func (dp Histogram) Add(in Histogram) Histogram { func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { switch { case dp.Timestamp() >= in.Timestamp(): - fail("out of order") + panic("out of order") case dp.Scale() != in.Scale(): - fail("scale changed") - case dp.ZeroCount() != in.ZeroCount(): - fail("zero count changed") + panic("scale changed") } - aggregate := func(dpBuckets, inBuckets pmetric.ExponentialHistogramDataPointBuckets) { - var ( - dp = Buckets{data: dpBuckets.BucketCounts(), offset: int(dpBuckets.Offset())} - in = Buckets{data: inBuckets.BucketCounts(), offset: int(inBuckets.Offset())} - aggr = Buckets{data: pcommon.NewUInt64Slice()} - ) - aggr.offset = int(min(dpBuckets.Offset(), inBuckets.Offset())) - if aggr.offset == dp.offset { - aggr.data = dp.data - } - aggr.EnsureLen(max(dp.Len(), in.Len())) - - for i := 0; i < aggr.Len(); i++ { - aggr.SetAt(i, dp.At(i)+in.At(i)) - } - - aggr.CopyTo(dpBuckets) + if dp.ZeroThreshold() != in.ZeroThreshold() { + hi, lo := expo.HiLo(dp, in, ExpHistogram.ZeroThreshold) + expo.RaiseZero(lo.ExponentialHistogramDataPoint, hi.ZeroThreshold()) } - aggregate(dp.Positive(), in.Positive()) - aggregate(dp.Negative(), in.Negative()) + expo.Merge(dp.Positive(), in.Positive()) + expo.Merge(dp.Negative(), in.Negative()) dp.SetTimestamp(in.Timestamp()) dp.SetCount(dp.Count() + in.Count()) + dp.SetZeroCount(dp.ZeroCount() + in.ZeroCount()) type T = ExpHistogram optionals := []field{ {get: T.Sum, set: T.SetSum, has: T.HasSum, del: T.RemoveSum, op: func(a, b float64) float64 { return a + b }}, - {get: T.Min, set: T.SetMin, has: T.HasMin, del: T.RemoveMin, op: func(a, b float64) float64 { return min(a, b) }}, - {get: T.Max, set: T.SetMax, has: T.HasMax, del: T.RemoveMax, op: func(a, b float64) float64 { return max(a, b) }}, + {get: T.Min, set: T.SetMin, has: T.HasMin, del: T.RemoveMin, op: math.Min}, + {get: T.Max, set: T.SetMax, has: T.HasMax, del: T.RemoveMax, op: math.Max}, } for _, f := range optionals { if f.has(dp) && f.has(in) { @@ -95,43 +73,9 @@ type field struct { op func(a, b float64) float64 } -type Buckets struct { - data pcommon.UInt64Slice - offset int -} - -func (o Buckets) Len() int { - return o.data.Len() + o.offset -} - -func (o Buckets) At(i int) uint64 { - idx, ok := o.idx(i) - if !ok { - return 0 +func pos(i int) int { + if i < 0 { + i = -i } - return o.data.At(idx) -} - -func (o Buckets) SetAt(i int, v uint64) { - idx, ok := o.idx(i) - if !ok { - return - } - o.data.SetAt(idx, v) -} - -func (o Buckets) EnsureLen(n int) { - sz := n - o.offset - o.data.EnsureCapacity(sz) - o.data.Append(make([]uint64, sz-o.data.Len())...) -} - -func (o Buckets) idx(i int) (int, bool) { - idx := i - o.offset - return idx, idx >= 0 && idx < o.data.Len() -} - -func (o Buckets) CopyTo(dst pmetric.ExponentialHistogramDataPointBuckets) { - o.data.CopyTo(dst.BucketCounts()) - dst.SetOffset(int32(o.offset)) + return i } diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go index 3adc46a30488..22a3d9eab1ca 100644 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -4,7 +4,9 @@ import ( "math" "testing" + "github.com/matryer/is" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" @@ -22,24 +24,24 @@ func TestAdd(t *testing.T) { cases := []tcase{{ name: "noop", - dp: expdp{ts: 0, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - in: expdp{ts: 1, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - want: expdp{ts: 1, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + dp: expdp{ts: 0, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + in: expdp{ts: 1, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + want: expdp{ts: 1, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, }, { name: "simple", - dp: expdp{ts: 0, pos: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - in: expdp{ts: 1, pos: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * 45}, - want: expdp{ts: 1, pos: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * (0 + 45)}, + dp: expdp{ts: 0, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, + in: expdp{ts: 1, pos: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * 45}, + want: expdp{ts: 1, pos: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * (0 + 45)}, }, { name: "lower+shorter", - dp: expdp{ts: 0, pos: buckets{ø, ø, ø, ø, ø, 1, 1, 1, 1}, neg: buckets{ø, ø, ø, ø, ø, 1, 1, 1, 1}, count: 2 * 4}, - in: expdp{ts: 1, pos: buckets{ø, ø, 1, 1, 1, 1, 1, ø, ø}, neg: buckets{ø, ø, 1, 1, 1, 1, 1, ø, ø}, count: 2 * 5}, - want: expdp{ts: 1, pos: buckets{ø, ø, 1, 1, 1, 2, 2, 1, 1}, neg: buckets{ø, ø, 1, 1, 1, 2, 2, 1, 1}, count: 2 * (4 + 5)}, + dp: expdp{ts: 0, pos: bins{ø, ø, ø, ø, ø, 1, 1, 1, 1}, neg: bins{ø, ø, ø, ø, ø, 1, 1, 1, 1}, count: 2 * 4}, + in: expdp{ts: 1, pos: bins{ø, ø, 1, 1, 1, 1, 1, ø, ø}, neg: bins{ø, ø, 1, 1, 1, 1, 1, ø, ø}, count: 2 * 5}, + want: expdp{ts: 1, pos: bins{ø, ø, 1, 1, 1, 2, 2, 1, 1}, neg: bins{ø, ø, 1, 1, 1, 2, 2, 1, 1}, count: 2 * (4 + 5)}, }, { name: "longer", - dp: expdp{ts: 0, pos: buckets{1, 1, 1, 1, 1, 1, ø, ø, ø}, neg: buckets{1, 1, 1, 1, 1, 1, ø, ø, ø}, count: 2 * 6}, - in: expdp{ts: 1, pos: buckets{1, 1, 1, 1, 1, 1, 1, 1, 1}, neg: buckets{1, 1, 1, 1, 1, 1, 1, 1, 1}, count: 2 * 9}, - want: expdp{ts: 1, pos: buckets{2, 2, 2, 2, 2, 2, 1, 1, 1}, neg: buckets{2, 2, 2, 2, 2, 2, 1, 1, 1}, count: 2 * (6 + 9)}, + dp: expdp{ts: 0, pos: bins{1, 1, 1, 1, 1, 1, ø, ø, ø}, neg: bins{1, 1, 1, 1, 1, 1, ø, ø, ø}, count: 2 * 6}, + in: expdp{ts: 1, pos: bins{1, 1, 1, 1, 1, 1, 1, 1, 1}, neg: bins{1, 1, 1, 1, 1, 1, 1, 1, 1}, count: 2 * 9}, + want: expdp{ts: 1, pos: bins{2, 2, 2, 2, 2, 2, 1, 1, 1}, neg: bins{2, 2, 2, 2, 2, 2, 1, 1, 1}, count: 2 * (6 + 9)}, }, { name: "optional/missing-dp", dp: expdp{ts: 0, pos: zeros(5).observe0(0.6, 2.4) /* */, count: 2}, @@ -55,6 +57,22 @@ func TestAdd(t *testing.T) { dp: expdp{ts: 0, pos: zeros(5).observe0(1.5, 5.3, 11.6) /* */, min: some(1.5), max: some(11.6), sum: some(18.4), count: 3}, in: expdp{ts: 1, pos: zeros(5).observe0(0.6, 3.3, 7.9) /* */, min: some(0.6), max: some(07.9), sum: some(11.8), count: 3}, want: expdp{ts: 1, pos: zeros(5).observe0(1.5, 5.3, 11.6, 0.6, 3.3, 7.9), min: some(0.6), max: some(11.6), sum: some(30.2), count: 6}, + }, { + name: "zero/count", + dp: expdp{ts: 0, pos: bins{1, 2}, zc: u64(3), count: 5}, + in: expdp{ts: 1, pos: bins{1, 0}, zc: u64(2), count: 3}, + want: expdp{ts: 1, pos: bins{2, 2}, zc: u64(5), count: 8}, + }, { + // (1, 2], (2, 4], (4, 8], (8, 16], (16, 32] + name: "zero/diff-simple", + dp: expdp{ts: 0, pos: bins{1, 1, 1}, zt: some(0.0), zc: u64(2)}, + in: expdp{ts: 1, pos: bins{ø, 1, 1}, zt: some(2.0), zc: u64(2)}, + want: expdp{ts: 1, pos: bins{ø, 2, 2}, zt: some(2.0), zc: u64(5)}, + }, { + name: "negative-offset", + dp: expdp{ts: 0, posb: some(buckets([]uint64{ /* */ 1, 2}, -2))}, + in: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3 /* */}, -5))}, + want: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3, 1, 2}, -5))}, }} for _, c := range cases { @@ -63,6 +81,9 @@ func TestAdd(t *testing.T) { want := c.want.into() got := dp.Add(in) + + is := is.NewRelaxed(t) + is.Equal(from(expo.Buckets(want.Positive())).String(), from(expo.Buckets(got.Positive())).String()) if err := pmetrictest.CompareExponentialHistogramDataPoint(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint); err != nil { t.Fatal(err) } @@ -73,18 +94,18 @@ func TestAdd(t *testing.T) { func TestBucketsIter(t *testing.T) { type tcase struct { name string - data Buckets + data expo.Buckets want []uint64 } cases := []tcase{{ name: "full", - data: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}.into(), - want: buckets{1, 2, 3, 4, 5, 6, 7, 8, 9}, + data: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}.into(), + want: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, }, { name: "3-6", - data: buckets{ø, ø, ø, 4, 5, 6, ø, ø, ø, ø}.into(), - want: buckets{0, 0, 0, 4, 5, 6}, + data: bins{ø, ø, ø, 4, 5, 6, ø, ø, ø, ø}.into(), + want: bins{0, 0, 0, 4, 5, 6}, }} for _, c := range cases { @@ -101,17 +122,17 @@ func TestBucketsIter(t *testing.T) { func TestBucketsSet(t *testing.T) { var ( - aggr = buckets{0, 0, 0, 0, 0, 0, 0, 0, 0}.into() - bkt0 = buckets{ø, 1, 1, 1, 1, ø, ø, ø, ø}.into() - bkt1 = buckets{ø, ø, ø, 1, 1, 1, 1, ø, ø}.into() - want = buckets{0, 1, 1, 2, 2, 1, 1, 0, 0} + aggr = bins{0, 0, 0, 0, 0, 0, 0, 0, 0}.into() + bkt0 = bins{ø, 1, 1, 1, 1, ø, ø, ø, ø}.into() + bkt1 = bins{ø, ø, ø, 1, 1, 1, 1, ø, ø}.into() + want = bins{0, 1, 1, 2, 2, 1, 1, 0, 0} ) for i := 0; i < aggr.Len(); i++ { aggr.SetAt(i, bkt0.At(i)+bkt1.At(i)) } - got := make(buckets, aggr.Len()) + got := make(bins, aggr.Len()) for i := 0; i < aggr.Len(); i++ { got[i] = aggr.At(i) } @@ -120,13 +141,13 @@ func TestBucketsSet(t *testing.T) { func TestBucketsEnsureLen(t *testing.T) { var ( - data = buckets{ø, ø, 3, 4} - want = buckets{0, 0, 3, 4, 0, 0, 0} + data = bins{ø, ø, 3, 4} + want = bins{0, 0, 3, 4, 0, 0, 0} ) bkt := data.into() bkt.EnsureLen(len(want)) - got := make(buckets, bkt.Len()) + got := make(bins, bkt.Len()) for i := 0; i < bkt.Len(); i++ { got[i] = bkt.At(i) } @@ -134,44 +155,59 @@ func TestBucketsEnsureLen(t *testing.T) { } type expdp struct { - ts int - pos buckets - neg buckets + ts int + + pos, neg bins + posb, negb *expo.Buckets scale int32 count uint64 sum *float64 min, max *float64 + + zc *uint64 + zt *float64 } func (e expdp) into() ExpHistogram { dp := pmetric.NewExponentialHistogramDataPoint() dp.SetTimestamp(pcommon.Timestamp(e.ts)) - pos := e.pos.into() - pos.data.CopyTo(dp.Positive().BucketCounts()) - dp.Positive().SetOffset(int32(pos.offset)) + posb := e.pos.into() + if e.posb != nil { + posb = *e.posb + } + pmetric.ExponentialHistogramDataPointBuckets(posb).CopyTo(dp.Positive()) - neg := e.neg.into() - neg.data.CopyTo(dp.Negative().BucketCounts()) - dp.Negative().SetOffset(int32(neg.offset)) + negb := e.neg.into() + if e.negb != nil { + negb = *e.negb + } + pmetric.ExponentialHistogramDataPointBuckets(negb).CopyTo(dp.Negative()) dp.SetScale(e.scale) dp.SetCount(e.count) - if e.sum != nil { - dp.SetSum(*e.sum) - } - if e.min != nil { - dp.SetMin(*e.min) - } - if e.max != nil { - dp.SetMax(*e.max) - } + + setnn(e.sum, dp.SetSum) + setnn(e.min, dp.SetMin) + setnn(e.max, dp.SetMax) + setnn(e.zc, dp.SetZeroCount) + setnn(e.zt, dp.SetZeroThreshold) return ExpHistogram{ExponentialHistogramDataPoint: dp} } +func setnn[T any](ptr *T, set func(T)) { + if ptr != nil { + set(*ptr) + } +} + func some[T any](v T) *T { return &v } + +func u64(v uint64) *uint64 { + return &v +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go new file mode 100644 index 000000000000..aef08c9a69e8 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go @@ -0,0 +1,86 @@ +package expo + +import ( + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +type Buckets pmetric.ExponentialHistogramDataPointBuckets + +func (b Buckets) At(i int) uint64 { + if idx, ok := b.idx(i); ok { + return b.data().At(idx) + } + return 0 +} + +func (b Buckets) SetAt(i int, v uint64) { + if idx, ok := b.idx(i); ok { + b.data().SetAt(idx, v) + } +} + +func (b Buckets) Len() int { + return b.data().Len() + int(b.Offset()) +} + +func (b Buckets) EnsureLen(n int) { + sz := n - int(b.Offset()) + b.data().EnsureCapacity(sz) + b.data().Append(make([]uint64, sz-b.data().Len())...) +} + +func (b Buckets) Truncate(first int) { + offset := first - b.Offset() + if b.Offset() >= offset { + return + } + + data := b.data().AsRaw()[offset-b.Offset():] + b.data().FromRaw(data) + b.as().SetOffset(int32(offset)) +} + +// Expand the buckets by n slots: +// - n < 0: prepend at front, lowering offset +// - n > 0: append to back +func (b Buckets) Expand(n int) { + switch { + case n < 0: + n = -n + us := pcommon.NewUInt64Slice() + us.Append(make([]uint64, n+b.data().Len())...) + for i := 0; i < b.data().Len(); i++ { + us.SetAt(i+n, b.data().At(i)) + } + us.MoveTo(b.data()) + b.as().SetOffset(int32(b.Offset() - n)) + case n > 0: + b.data().Append(make([]uint64, n)...) + } +} + +func (b Buckets) Offset() int { + return int(b.as().Offset()) +} + +func (b Buckets) idx(i int) (int, bool) { + idx := i - b.Offset() + return idx, idx >= 0 && idx < b.data().Len() +} + +func (b Buckets) data() pcommon.UInt64Slice { + return b.as().BucketCounts() +} + +func (b Buckets) as() pmetric.ExponentialHistogramDataPointBuckets { + return pmetric.ExponentialHistogramDataPointBuckets(b) +} + +func HiLo[T any, N int | float64](a, b T, fn func(T) N) (hi, lo T) { + an, bn := fn(a), fn(b) + if an > bn { + return a, b + } + return b, a +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go new file mode 100644 index 000000000000..6789aa71a596 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -0,0 +1,41 @@ +package expotest + +import ( + "math" + + "go.opentelemetry.io/collector/pdata/pmetric" +) + +const ( + Empty = math.MaxUint64 + ø = Empty +) + +// idx: 0 1 2 3 4 5 6 7 +// bkt: -3 -2 -1 0 1 2 3 4 +type Bins [8]uint64 + +func Buckets(bins Bins) pmetric.ExponentialHistogramDataPointBuckets { + start := 0 + for i := 0; i < len(bins); i++ { + if bins[i] != ø { + start = i + break + } + } + + end := len(bins) + for i := start; i < len(bins); i++ { + if bins[i] == ø { + end = i + break + } + } + + counts := bins[start:end] + + buckets := pmetric.NewExponentialHistogramDataPointBuckets() + buckets.SetOffset(int32(start - 3)) + buckets.BucketCounts().FromRaw(counts) + return buckets +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go new file mode 100644 index 000000000000..b4176bca93f2 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -0,0 +1,51 @@ +package expo + +import ( + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +func Merge(arel, brel pmetric.ExponentialHistogramDataPointBuckets) { + a, b := Absolute{buckets: arel}, Absolute{buckets: brel} + + lo := min(a.Lower(), b.Lower()) + up := max(a.Upper(), b.Upper()) + + size := up - lo + + counts := pcommon.NewUInt64Slice() + counts.Append(make([]uint64, size-counts.Len())...) + + for i := 0; i < counts.Len(); i++ { + counts.SetAt(i, a.Abs(lo+i)+b.Abs(lo+i)) + } + + a.SetOffset(int32(lo)) + counts.MoveTo(a.BucketCounts()) +} + +type buckets = pmetric.ExponentialHistogramDataPointBuckets + +type Absolute struct { + buckets +} + +func (a Absolute) Abs(at int) uint64 { + if i, ok := a.idx(at); ok { + return a.BucketCounts().At(i) + } + return 0 +} + +func (a Absolute) Upper() int { + return a.BucketCounts().Len() + int(a.Offset()) +} + +func (a Absolute) Lower() int { + return int(a.Offset()) +} + +func (a Absolute) idx(at int) (int, bool) { + idx := at - a.Lower() + return idx, idx >= 0 && idx < a.BucketCounts().Len() +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go new file mode 100644 index 000000000000..5539a5b0c5bd --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go @@ -0,0 +1,52 @@ +package expo_test + +import ( + "fmt" + "testing" + + "github.com/matryer/is" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" +) + +const ø = expotest.Empty + +type bins = expotest.Bins + +func TestMerge(t *testing.T) { + cases := []struct { + a, b bins + want bins + }{{ + // -3 -2 -1 0 1 2 3 4 + a: bins{ø, ø, ø, ø, ø, ø, ø, ø}, + b: bins{ø, ø, ø, ø, ø, ø, ø, ø}, + want: bins{ø, ø, ø, ø, ø, ø, ø, ø}, + }, { + a: bins{ø, ø, 1, 1, 1, ø, ø, ø}, + b: bins{ø, 1, 1, ø, ø, ø, ø, ø}, + want: bins{ø, 1, 2, 1, 1, ø, ø, ø}, + }, { + a: bins{ø, ø, ø, ø, 1, 1, 1, ø}, + b: bins{ø, ø, ø, ø, 1, 1, 1, ø}, + want: bins{ø, ø, ø, ø, 2, 2, 2, ø}, + }, { + a: bins{ø, 1, 1, ø, ø, ø, ø, ø}, + b: bins{ø, ø, ø, ø, 1, 1, ø, ø}, + want: bins{ø, 1, 1, 0, 1, 1, ø, ø}, + }} + + for _, cs := range cases { + a := expotest.Buckets(cs.a) + b := expotest.Buckets(cs.b) + want := expotest.Buckets(cs.want) + + name := fmt.Sprintf("(%+d,%d)+(%+d,%d)=(%+d,%d)", a.Offset(), a.BucketCounts().Len(), b.Offset(), b.BucketCounts().Len(), want.Offset(), want.BucketCounts().Len()) + t.Run(name, func(t *testing.T) { + expo.Merge(a, b) + is := is.NewRelaxed(t) + is.Equal(want.Offset(), a.Offset()) + is.Equal(want.BucketCounts().AsRaw(), a.BucketCounts().AsRaw()) + }) + } +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go new file mode 100644 index 000000000000..620627f49387 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -0,0 +1,18 @@ +package expo + +import "math" + +type Scale int32 + +// https://opentelemetry.io/docs/specs/otel/metrics/data-model/#all-scales-use-the-logarithm-function +func (scale Scale) Idx(v float64) int { + // Special case for power-of-two values. + if frac, exp := math.Frexp(v); frac == 0.5 { + return ((exp - 1) << scale) - 1 + } + + scaleFactor := math.Ldexp(math.Log2E, int(scale)) + // Note: math.Floor(value) equals math.Ceil(value)-1 when value + // is not a power of two, which is checked above. + return int(math.Floor(math.Log(v) * scaleFactor)) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go new file mode 100644 index 000000000000..f07ab2933d79 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -0,0 +1,27 @@ +package expo + +import "go.opentelemetry.io/collector/pdata/pmetric" + +type DataPoint = pmetric.ExponentialHistogramDataPoint + +func RaiseZero(dp DataPoint, threshold float64) { + if dp.ZeroThreshold() > threshold { + panic("new threshold must be greater") + } + + scale := Scale(dp.Scale()) + idx := scale.Idx(threshold) + + move := func(bin pmetric.ExponentialHistogramDataPointBuckets) { + bkt := Buckets(bin) + for i := 0; i < idx; i++ { + n := bkt.At(i) + dp.SetZeroCount(dp.ZeroCount() + n) + bkt.SetAt(i, 0) + } + bkt.Truncate(idx) + } + + move(dp.Positive()) + move(dp.Negative()) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/helper_test.go b/processor/deltatocumulativeprocessor/internal/data/helper_test.go index daa0753d373c..58b08ed102f4 100644 --- a/processor/deltatocumulativeprocessor/internal/data/helper_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/helper_test.go @@ -3,84 +3,131 @@ package data import ( "math" "sort" + "strconv" + "strings" "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" ) -type buckets []uint64 +type bins []uint64 -func (b buckets) into() Buckets { +func (bins bins) into() expo.Buckets { + off, counts := bins.split() + return buckets(counts, offset(len(off))) +} + +func (bins bins) split() (offset, counts []uint64) { start := 0 - for i := 0; i < len(b); i++ { - if b[i] != ø { + for i := 0; i < len(bins); i++ { + if bins[i] != ø { start = i break } } - end := len(b) - for i := start; i < len(b); i++ { - if b[i] == ø { + end := len(bins) + for i := start; i < len(bins); i++ { + if bins[i] == ø { end = i break } } - data := pcommon.NewUInt64Slice() - data.FromRaw([]uint64(b[start:end])) - return Buckets{ - data: data, - offset: start, + return bins[0:start], bins[start:end] +} + +func (bins bins) String() string { + var b strings.Builder + b.WriteString("[") + for i, v := range bins { + if i != 0 { + b.WriteString(", ") + } + if v == ø { + b.WriteString("_") + continue + } + b.WriteString(strconv.FormatUint(v, 10)) } + b.WriteString("]") + return b.String() } -func TestBucketsHelper(t *testing.T) { - uints := func(data ...uint64) pcommon.UInt64Slice { - us := pcommon.NewUInt64Slice() - us.FromRaw(data) - return us +func from(buckets expo.Buckets) bins { + counts := pmetric.ExponentialHistogramDataPointBuckets(buckets).BucketCounts() + off := buckets.Offset() + if off < 0 { + off = -off + } + bs := make(bins, counts.Len()+off) + for i := 0; i < off; i++ { + bs[i] = ø } + for i := 0; i < counts.Len(); i++ { + bs[i+off] = counts.At(i) + } + return bs +} +func TestBucketsHelper(t *testing.T) { cases := []struct { - bkts buckets - want Buckets + bins bins + want expo.Buckets }{{ - bkts: buckets{}, - want: Buckets{data: uints(), offset: 0}, + bins: bins{}, + want: buckets(nil, offset(0)), }, { - bkts: buckets{1, 2, 3, 4}, - want: Buckets{data: uints(1, 2, 3, 4), offset: 0}, + bins: bins{1, 2, 3, 4}, + want: buckets([]uint64{1, 2, 3, 4}, offset(0)), }, { - bkts: buckets{ø, ø, 3, 4}, - want: Buckets{data: uints(3, 4), offset: 2}, + bins: bins{ø, ø, 3, 4}, + want: buckets([]uint64{3, 4}, offset(2)), }, { - bkts: buckets{ø, ø, 3, 4, ø, ø}, - want: Buckets{data: uints(3, 4), offset: 2}, + bins: bins{ø, ø, 3, 4, ø, ø}, + want: buckets([]uint64{3, 4}, offset(2)), }, { - bkts: buckets{1, 2, ø, ø}, - want: Buckets{data: uints(1, 2), offset: 0}, + bins: bins{1, 2, ø, ø}, + want: buckets([]uint64{1, 2}, offset(0)), }} for _, c := range cases { - got := c.bkts.into() + got := c.bins.into() require.Equal(t, c.want, got) } } -func zeros(size int) buckets { - return make(buckets, size) +type offset int + +func buckets(data []uint64, offset offset) expo.Buckets { + if data == nil { + data = make([]uint64, 0) + } + bs := pmetric.NewExponentialHistogramDataPointBuckets() + bs.BucketCounts().FromRaw(data) + bs.SetOffset(int32(offset)) + return expo.Buckets(bs) +} + +func zeros(size int) bins { + return make(bins, size) } // observe some points with scale 0 -func (dps buckets) observe0(pts ...float64) buckets { +func (dps bins) observe0(pts ...float64) bins { if len(pts) == 0 { return dps } + // https://opentelemetry.io/docs/specs/otel/metrics/data-model/#scale-zero-extract-the-exponent idx := func(v float64) int { - return int(math.Ceil(math.Log2(v))) + frac, exp := math.Frexp(v) + if frac == 0.5 { + exp-- + } + return exp } sort.Float64s(pts) @@ -101,13 +148,13 @@ func (dps buckets) observe0(pts ...float64) buckets { func TestObserve0(t *testing.T) { cases := []struct { pts []float64 - want buckets + want bins }{{ pts: []float64{1.5, 5.3, 11.6}, - want: buckets{0, 1, 0, 1, 1}, + want: bins{0, 1, 0, 1, 1}, }, { pts: []float64{0.6, 3.3, 7.9}, - want: buckets{1, 0, 1, 1, 0}, + want: bins{1, 0, 1, 1, 0}, }} for _, c := range cases { From 918c168c7994d0a9409b02288e5c6233d5341b8f Mon Sep 17 00:00:00 2001 From: sh0rez Date: Thu, 18 Apr 2024 13:47:57 +0200 Subject: [PATCH 04/24] data: expo.WidenZero, adjust zero-count implements zero-bucket widening to add two datapoints of different zero count --- .../internal/data/add.go | 2 +- .../internal/data/exphist_test.go | 2 +- .../internal/data/expo/expotest/bins.go | 5 +- .../internal/data/expo/expotest/equal.go | 34 +++++++ .../internal/data/expo/expotest/histogram.go | 36 +++++++ .../internal/data/expo/merge.go | 6 +- .../internal/data/expo/merge_test.go | 6 +- .../internal/data/expo/scale.go | 12 +++ .../internal/data/expo/zero.go | 57 ++++++++--- .../internal/data/expo/zero_test.go | 99 +++++++++++++++++++ 10 files changed, 235 insertions(+), 24 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 80be60831690..327af935c823 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -38,7 +38,7 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { if dp.ZeroThreshold() != in.ZeroThreshold() { hi, lo := expo.HiLo(dp, in, ExpHistogram.ZeroThreshold) - expo.RaiseZero(lo.ExponentialHistogramDataPoint, hi.ZeroThreshold()) + expo.WidenZero(lo.ExponentialHistogramDataPoint, hi.ZeroThreshold()) } expo.Merge(dp.Positive(), in.Positive()) diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go index 22a3d9eab1ca..ab218c005f55 100644 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -67,7 +67,7 @@ func TestAdd(t *testing.T) { name: "zero/diff-simple", dp: expdp{ts: 0, pos: bins{1, 1, 1}, zt: some(0.0), zc: u64(2)}, in: expdp{ts: 1, pos: bins{ø, 1, 1}, zt: some(2.0), zc: u64(2)}, - want: expdp{ts: 1, pos: bins{ø, 2, 2}, zt: some(2.0), zc: u64(5)}, + want: expdp{ts: 1, pos: bins{ø, 2, 2}, zt: some(2.0), zc: u64((2 + 2) + 1)}, }, { name: "negative-offset", dp: expdp{ts: 0, posb: some(buckets([]uint64{ /* */ 1, 2}, -2))}, diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go index 6789aa71a596..f89bd0a9f0c5 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -11,8 +11,9 @@ const ( ø = Empty ) -// idx: 0 1 2 3 4 5 6 7 -// bkt: -3 -2 -1 0 1 2 3 4 +// index: 0 1 2 3 4 5 6 7 +// bucket: -3 -2 -1 0 1 2 3 4 +// bounds: (0.125,0.25], (0.25,0.5], (0.5,1], (1,2], (2,4], (4,8], (8,16], (16,32] type Bins [8]uint64 func Buckets(bins Bins) pmetric.ExponentialHistogramDataPointBuckets { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go new file mode 100644 index 000000000000..6fa68661dce8 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -0,0 +1,34 @@ +package expotest + +import ( + "testing" + + "github.com/matryer/is" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +type I struct { + *is.I +} + +func Is(t *testing.T) *I { + return &I{I: is.NewRelaxed(t)} +} + +func (is *I) Equal(want, got any) { + switch got := got.(type) { + case expo.DataPoint: + want := want.(expo.DataPoint) + is.I.Equal(want.ZeroCount(), got.ZeroCount()) // zero-count + is.I.Equal(want.ZeroThreshold(), got.ZeroThreshold()) // zero-threshold + is.Equal(want.Positive(), got.Positive()) + is.Equal(want.Negative(), got.Negative()) + case pmetric.ExponentialHistogramDataPointBuckets: + want := want.(pmetric.ExponentialHistogramDataPointBuckets) + is.I.Equal(want.Offset(), got.Offset()) // offset + is.I.Equal(want.BucketCounts().AsRaw(), got.BucketCounts().AsRaw()) // counts + default: + is.I.Equal(got, want) + } +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go new file mode 100644 index 000000000000..19bb979697a6 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go @@ -0,0 +1,36 @@ +package expotest + +import ( + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +type Histogram struct { + Pos, Neg Bins + PosNeg Bins + + Scale int + + Zt float64 + Zc uint64 +} + +func DataPoint(hist Histogram) expo.DataPoint { + dp := pmetric.NewExponentialHistogramDataPoint() + + posneg := Buckets(hist.PosNeg) + posneg.CopyTo(dp.Positive()) + posneg.CopyTo(dp.Negative()) + + if (hist.Pos != Bins{}) { + Buckets(hist.Pos).MoveTo(dp.Positive()) + } + if (hist.Neg != Bins{}) { + Buckets(hist.Neg).MoveTo(dp.Negative()) + } + + dp.SetScale(int32(hist.Scale)) + dp.SetZeroThreshold(hist.Zt) + dp.SetZeroCount(hist.Zc) + return dp +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index b4176bca93f2..0bf50c7d3408 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -6,7 +6,7 @@ import ( ) func Merge(arel, brel pmetric.ExponentialHistogramDataPointBuckets) { - a, b := Absolute{buckets: arel}, Absolute{buckets: brel} + a, b := Abs(arel), Abs(brel) lo := min(a.Lower(), b.Lower()) up := max(a.Upper(), b.Upper()) @@ -24,6 +24,10 @@ func Merge(arel, brel pmetric.ExponentialHistogramDataPointBuckets) { counts.MoveTo(a.BucketCounts()) } +func Abs(buckets buckets) Absolute { + return Absolute{buckets: buckets} +} + type buckets = pmetric.ExponentialHistogramDataPointBuckets type Absolute struct { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go index 5539a5b0c5bd..06f3855fe343 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go @@ -4,7 +4,6 @@ import ( "fmt" "testing" - "github.com/matryer/is" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" ) @@ -44,9 +43,8 @@ func TestMerge(t *testing.T) { name := fmt.Sprintf("(%+d,%d)+(%+d,%d)=(%+d,%d)", a.Offset(), a.BucketCounts().Len(), b.Offset(), b.BucketCounts().Len(), want.Offset(), want.BucketCounts().Len()) t.Run(name, func(t *testing.T) { expo.Merge(a, b) - is := is.NewRelaxed(t) - is.Equal(want.Offset(), a.Offset()) - is.Equal(want.BucketCounts().AsRaw(), a.BucketCounts().AsRaw()) + is := expotest.Is(t) + is.Equal(want, a) }) } } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index 620627f49387..8b975dc92da7 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -16,3 +16,15 @@ func (scale Scale) Idx(v float64) int { // is not a power of two, which is checked above. return int(math.Floor(math.Log(v) * scaleFactor)) } + +// Bounds returns the half-open interval (min,max] of the bucket at index. +// This means a value min threshold { - panic("new threshold must be greater") +// WidenZero widens the zero-bucket to span at least [-width,width], possibly wider +// if min falls in the middle of a bucket +func WidenZero(dp DataPoint, width float64) { + switch { + case width == dp.ZeroThreshold(): + return + case width < dp.ZeroThreshold(): + panic(fmt.Sprintf("min must be larger than current threshold (%f)", dp.ZeroThreshold())) } scale := Scale(dp.Scale()) - idx := scale.Idx(threshold) - - move := func(bin pmetric.ExponentialHistogramDataPointBuckets) { - bkt := Buckets(bin) - for i := 0; i < idx; i++ { - n := bkt.At(i) - dp.SetZeroCount(dp.ZeroCount() + n) - bkt.SetAt(i, 0) + lo := scale.Idx(width) + + widen := func(bs pmetric.ExponentialHistogramDataPointBuckets) { + abs := Abs(bs) + for i := abs.Lower(); i <= lo; i++ { + dp.SetZeroCount(dp.ZeroCount() + abs.Abs(i)) } - bkt.Truncate(idx) + up := abs.Upper() + abs.Slice(min(lo+1, up), up) + } + + widen(dp.Positive()) + widen(dp.Negative()) + + _, max := scale.Bounds(lo) + dp.SetZeroThreshold(max) +} + +func (a Absolute) Slice(from, to int) { + lo, up := a.Lower(), a.Upper() + switch { + case from > to: + panic(fmt.Sprintf("bad bounds: must be from<=to (got %d<=%d)", from, to)) + case from < lo || to > up: + panic(fmt.Sprintf("%d:%d is out of bounds for %d:%d", from, to, lo, up)) } - move(dp.Positive()) - move(dp.Negative()) + first := from - lo + last := to - lo + + a.BucketCounts().FromRaw(a.BucketCounts().AsRaw()[first:last]) + a.SetOffset(int32(from)) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go new file mode 100644 index 000000000000..da2af840c013 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -0,0 +1,99 @@ +package expo_test + +import ( + "fmt" + "testing" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" +) + +type hist = expotest.Histogram + +func TestWidenZero(t *testing.T) { + cases := []struct { + name string + hist hist + want hist + min float64 + }{{ + // -3 -2 -1 0 1 2 3 4 + // (0.125,0.25], (0.25,0.5], (0.5,1], (1,2], (2,4], (4,8], (8,16], (16,32] + // + // -3 -2 -1 0 1 2 3 4 + hist: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}, Zt: 0, Zc: 0}, + want: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}, Zt: 0, Zc: 0}, + }, { + // zt=2 is upper boundary of bucket 0. keep buckets [1:n] + hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}, Zt: 0, Zc: 2}, + want: hist{PosNeg: bins{ø, ø, ø, ø, 3, 4, 5, ø}, Zt: 2, Zc: 2 + 2*(1+2)}, + }, { + // zt=3 is within bucket 1. keep buckets [2:n] + // set zt=4 because it must cover full buckets + hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}, Zt: 0, Zc: 2}, + min: 3, + want: hist{PosNeg: bins{ø, ø, ø, ø, ø, 4, 5, ø}, Zt: 4, Zc: 2 + 2*(1+2+3)}, + }} + + for _, cs := range cases { + name := fmt.Sprintf("%.2f->%.2f", cs.hist.Zt, cs.want.Zt) + t.Run(name, func(t *testing.T) { + hist := expotest.DataPoint(cs.hist) + want := expotest.DataPoint(cs.want) + + zt := cs.min + if zt == 0 { + zt = want.ZeroThreshold() + } + expo.WidenZero(hist, zt) + + is := expotest.Is(t) + is.Equal(want, hist) + }) + } +} + +func TestSlice(t *testing.T) { + cases := []struct { + bins bins + want bins + }{{ + // -3 -2 -1 0 1 2 3 4 + bins: bins{ø, ø, ø, ø, ø, ø, ø, ø}, + want: bins{ø, ø, ø, ø, ø, ø, ø, ø}, + }, { + bins: bins{1, 2, 3, 4, 5, 6, 7, 8}, + want: bins{1, 2, 3, 4, 5, 6, 7, 8}, + }, { + bins: bins{ø, 2, 3, 4, 5, 6, 7, ø}, + want: bins{ø, ø, 3, 4, 5, ø, ø, ø}, + }} + + for _, cs := range cases { + from, to := 0, len(cs.want) + for i := 0; i < len(cs.want); i++ { + if cs.want[i] != ø { + from += i + break + } + } + for i := from; i < len(cs.want); i++ { + if cs.want[i] == ø { + to = i + break + } + } + from -= 3 + to -= 3 + + t.Run(fmt.Sprintf("[%d:%d]", from, to), func(t *testing.T) { + bins := expotest.Buckets(cs.bins) + want := expotest.Buckets(cs.want) + + expo.Abs(bins).Slice(from, to) + + is := expotest.Is(t) + is.Equal(want, bins) + }) + } +} From 85fec7e7b94d3134e2b7999ea766034a3c6fdc0f Mon Sep 17 00:00:00 2001 From: sh0rez Date: Thu, 18 Apr 2024 23:24:23 +0200 Subject: [PATCH 05/24] data: expo.Downscale, adjust scale if neccessary --- .../internal/data/add.go | 19 +++-- .../internal/data/exphist_test.go | 18 ++++- .../internal/data/expo/buckets.go | 8 +- .../internal/data/expo/expotest/bins.go | 35 ++++++++ .../internal/data/expo/merge.go | 13 +++ .../internal/data/expo/scale.go | 65 ++++++++++++++- .../internal/data/expo/scale_test.go | 79 +++++++++++++++++++ .../internal/data/expo/zero.go | 5 +- 8 files changed, 228 insertions(+), 14 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 327af935c823..5da87608afc7 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -29,11 +29,19 @@ func (dp Histogram) Add(in Histogram) Histogram { } func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { + type H = ExpHistogram + switch { case dp.Timestamp() >= in.Timestamp(): panic("out of order") - case dp.Scale() != in.Scale(): - panic("scale changed") + } + + if dp.Scale() != in.Scale() { + hi, lo := expo.HiLo(dp, in, H.Scale) + from, to := expo.Scale(hi.Scale()), expo.Scale(lo.Scale()) + expo.Downscale(hi.Positive(), from, to) + expo.Downscale(hi.Negative(), from, to) + hi.SetScale(lo.Scale()) } if dp.ZeroThreshold() != in.ZeroThreshold() { @@ -48,11 +56,10 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { dp.SetCount(dp.Count() + in.Count()) dp.SetZeroCount(dp.ZeroCount() + in.ZeroCount()) - type T = ExpHistogram optionals := []field{ - {get: T.Sum, set: T.SetSum, has: T.HasSum, del: T.RemoveSum, op: func(a, b float64) float64 { return a + b }}, - {get: T.Min, set: T.SetMin, has: T.HasMin, del: T.RemoveMin, op: math.Min}, - {get: T.Max, set: T.SetMax, has: T.HasMax, del: T.RemoveMax, op: math.Max}, + {get: H.Sum, set: H.SetSum, has: H.HasSum, del: H.RemoveSum, op: func(a, b float64) float64 { return a + b }}, + {get: H.Min, set: H.SetMin, has: H.HasMin, del: H.RemoveMin, op: math.Min}, + {get: H.Max, set: H.SetMax, has: H.HasMax, del: H.RemoveMax, op: math.Max}, } for _, f := range optionals { if f.has(dp) && f.has(in) { diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go index ab218c005f55..578616fe9291 100644 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -4,9 +4,9 @@ import ( "math" "testing" - "github.com/matryer/is" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" @@ -73,6 +73,17 @@ func TestAdd(t *testing.T) { dp: expdp{ts: 0, posb: some(buckets([]uint64{ /* */ 1, 2}, -2))}, in: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3 /* */}, -5))}, want: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3, 1, 2}, -5))}, + }, { + name: "scale/diff", + dp: expdp{ts: 0, posb: some(expo.Buckets(expotest.Observe(expo.Scale(1), 1, 2, 3, 4))), scale: 1}, + in: expdp{ts: 1, posb: some(expo.Buckets(expotest.Observe(expo.Scale(0), 1, 2, 3, 4))), scale: 0}, + want: expdp{ts: 1, scale: 0, posb: func() *expo.Buckets { + bs := pmetric.NewExponentialHistogramDataPointBuckets() + expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) + expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) + bs.BucketCounts().Append([]uint64{0, 0}...) // rescaling leaves zeroed memory. this is expected + return some(expo.Buckets(bs)) + }()}, }} for _, c := range cases { @@ -82,8 +93,9 @@ func TestAdd(t *testing.T) { got := dp.Add(in) - is := is.NewRelaxed(t) - is.Equal(from(expo.Buckets(want.Positive())).String(), from(expo.Buckets(got.Positive())).String()) + is := expotest.Is(t) + is.Equal(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint) + if err := pmetrictest.CompareExponentialHistogramDataPoint(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint); err != nil { t.Fatal(err) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go index aef08c9a69e8..a8d02df65b39 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go @@ -77,7 +77,13 @@ func (b Buckets) as() pmetric.ExponentialHistogramDataPointBuckets { return pmetric.ExponentialHistogramDataPointBuckets(b) } -func HiLo[T any, N int | float64](a, b T, fn func(T) N) (hi, lo T) { +type ord interface { + int | int32 | float64 +} + +// HiLo returns the greater of a and b by comparing the result of applying fn to +// each +func HiLo[T any, N ord](a, b T, fn func(T) N) (hi, lo T) { an, bn := fn(a), fn(b) if an > bn { return a, b diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go index f89bd0a9f0c5..5e9a3a284ddd 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -1,8 +1,10 @@ package expotest import ( + "fmt" "math" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" ) @@ -40,3 +42,36 @@ func Buckets(bins Bins) pmetric.ExponentialHistogramDataPointBuckets { buckets.BucketCounts().FromRaw(counts) return buckets } + +func ObserveInto(bs pmetric.ExponentialHistogramDataPointBuckets, scale expo.Scale, pts ...float64) { + counts := bs.BucketCounts() + + for _, pt := range pts { + pt = math.Abs(pt) + if pt <= 0.125 || pt > 32 { + panic(fmt.Sprintf("out of bounds: 0.125 < %f <= 32", pt)) + } + + idx := scale.Idx(pt) - int(bs.Offset()) + switch { + case idx < 0: + bs.SetOffset(bs.Offset() + int32(idx)) + counts.FromRaw(append(make([]uint64, -idx), counts.AsRaw()...)) + idx = 0 + case idx >= counts.Len(): + counts.Append(make([]uint64, idx-counts.Len()+1)...) + } + + counts.SetAt(idx, counts.At(idx)+1) + } +} + +func Observe(scale expo.Scale, pts ...float64) pmetric.ExponentialHistogramDataPointBuckets { + bs := pmetric.NewExponentialHistogramDataPointBuckets() + ObserveInto(bs, scale, pts...) + return bs +} + +func Observe0(pts ...float64) pmetric.ExponentialHistogramDataPointBuckets { + return Observe(0, pts...) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index 0bf50c7d3408..84e3ac604038 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -5,6 +5,8 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" ) +// Merge combines the counts of buckets a and b into a. +// Both buckets MUST be of same scale func Merge(arel, brel pmetric.ExponentialHistogramDataPointBuckets) { a, b := Abs(arel), Abs(brel) @@ -30,10 +32,19 @@ func Abs(buckets buckets) Absolute { type buckets = pmetric.ExponentialHistogramDataPointBuckets +// Absolute addresses bucket counts using an absolute scale, such that the +// following holds true: +// +// for i := range counts: Scale(…).Idx(counts[i]) == i +// +// It spans from [[Absolute.Lower]:[Absolute.Upper]] type Absolute struct { buckets } +// Abs returns the value at absolute index 'at'. The following holds true: +// +// Scale(…).Idx(At(i)) == i func (a Absolute) Abs(at int) uint64 { if i, ok := a.idx(at); ok { return a.BucketCounts().At(i) @@ -41,10 +52,12 @@ func (a Absolute) Abs(at int) uint64 { return 0 } +// Upper returns the minimal index outside the set, such that every i < Upper func (a Absolute) Upper() int { return a.BucketCounts().Len() + int(a.Offset()) } +// Lower returns the minimal index inside the set, such that every i >= Lower func (a Absolute) Lower() int { return int(a.Offset()) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index 8b975dc92da7..c3ea306e7f73 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -1,11 +1,17 @@ package expo -import "math" +import ( + "math" + + "go.opentelemetry.io/collector/pdata/pmetric" +) type Scale int32 -// https://opentelemetry.io/docs/specs/otel/metrics/data-model/#all-scales-use-the-logarithm-function +// Idx gives the bucket index v belongs into func (scale Scale) Idx(v float64) int { + // from: https://opentelemetry.io/docs/specs/otel/metrics/data-model/#all-scales-use-the-logarithm-function + // Special case for power-of-two values. if frac, exp := math.Frexp(v); frac == 0.5 { return ((exp - 1) << scale) - 1 @@ -18,9 +24,11 @@ func (scale Scale) Idx(v float64) int { } // Bounds returns the half-open interval (min,max] of the bucket at index. -// This means a value min to; at-- { + Collapse(bs) + } +} + +// Collapse merges adjacent buckets and zeros the remaining area: +// +// before: 1 1 1 1 1 1 1 1 1 1 1 1 +// after: 2 2 2 2 2 2 0 0 0 0 0 0 +func Collapse(bs pmetric.ExponentialHistogramDataPointBuckets) { + counts := bs.BucketCounts() + size := counts.Len() / 2 + if counts.Len()%2 != 0 { + size++ + } + + shift := 0 + if bs.Offset()%2 != 0 { + bs.SetOffset(bs.Offset() - 1) + shift-- + } + bs.SetOffset(bs.Offset() / 2) + + for i := 0; i < size; i++ { + k := i*2 + shift + + if i == 0 && k == -1 { + counts.SetAt(i, counts.At(k+1)) + continue + } + + counts.SetAt(i, counts.At(k)) + if k+1 < counts.Len() { + counts.SetAt(i, counts.At(k)+counts.At(k+1)) + } + } + + for i := size; i < counts.Len(); i++ { + counts.SetAt(i, 0) + } +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go new file mode 100644 index 000000000000..4cd3ae536aa1 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -0,0 +1,79 @@ +package expo_test + +import ( + "fmt" + "strconv" + "strings" + "testing" + + "github.com/matryer/is" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +func TestDownscale(t *testing.T) { + type Repr[T any] struct { + scale expo.Scale + bkt T + } + + cases := [][]Repr[string]{{ + {scale: 2, bkt: "1 1 1 1 1 1 1 1 1 1 1 1"}, + {scale: 1, bkt: " 2 2 2 2 2 2 "}, + {scale: 0, bkt: " 4 4 4 "}, + }, { + {scale: 2, bkt: "ø 1 1 1 1 1 1 1 1 1 1 1"}, + {scale: 1, bkt: " 1 2 2 2 2 2 "}, + {scale: 0, bkt: " 3 4 4 "}, + }, { + {scale: 2, bkt: "ø ø 1 1 1 1 1 1 1 1 1 1"}, + {scale: 1, bkt: " ø 2 2 2 2 2 "}, + {scale: 0, bkt: " 2 4 4 "}, + }, { + {scale: 2, bkt: "ø ø ø ø 1 1 1 1 1 1 1 1"}, + {scale: 1, bkt: " ø ø 2 2 2 2 "}, + {scale: 0, bkt: " ø 4 4 "}, + }, { + {scale: 2, bkt: "1 1 1 1 1 1 1 1 1 "}, + {scale: 1, bkt: " 2 2 2 2 1 "}, + {scale: 0, bkt: " 4 4 1 "}, + }, { + {scale: 2, bkt: "1 1 1 1 1 1 1 1 1 1 1 1"}, + {scale: 0, bkt: " 4 4 4 "}, + }} + + type B = pmetric.ExponentialHistogramDataPointBuckets + for i, reprs := range cases { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + buckets := make([]Repr[B], len(reprs)) + for i, r := range reprs { + bkt := pmetric.NewExponentialHistogramDataPointBuckets() + for _, elem := range strings.Fields(r.bkt) { + if elem == "ø" { + bkt.SetOffset(bkt.Offset() + 1) + continue + } + n, err := strconv.Atoi(elem) + if err != nil { + panic(err) + } + bkt.BucketCounts().Append(uint64(n)) + } + buckets[i] = Repr[B]{scale: r.scale, bkt: bkt} + } + + is := is.NewRelaxed(t) + for i := 0; i < len(buckets)-1; i++ { + expo.Downscale(buckets[i].bkt, buckets[i].scale, buckets[i+1].scale) + + is.Equal(buckets[i+1].bkt.Offset(), buckets[i].bkt.Offset()) // offset must be equal + + want := buckets[i+1].bkt.BucketCounts().AsRaw() + got := buckets[i].bkt.BucketCounts().AsRaw() + + is.Equal(want, got[:len(want)]) // counts must be equal + is.Equal(make([]uint64, len(got)-len(want)), got[len(want):]) // extra space must be zero + } + }) + } +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go index df187882716e..14095e219ec9 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -9,7 +9,9 @@ import ( type DataPoint = pmetric.ExponentialHistogramDataPoint // WidenZero widens the zero-bucket to span at least [-width,width], possibly wider -// if min falls in the middle of a bucket +// if min falls in the middle of a bucket. +// +// Both buckets MUST be of same scale. func WidenZero(dp DataPoint, width float64) { switch { case width == dp.ZeroThreshold(): @@ -37,6 +39,7 @@ func WidenZero(dp DataPoint, width float64) { dp.SetZeroThreshold(max) } +// Slice drops data outside the range from <= i < to from the bucket counts. It behaves the same as Go's [a:b] func (a Absolute) Slice(from, to int) { lo, up := a.Lower(), a.Upper() switch { From 943359e1b3ea5d456917bcbb9f5ad9cfb86a7175 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Thu, 18 Apr 2024 23:29:26 +0200 Subject: [PATCH 06/24] *: addlicense, lint, sort imports --- .../internal/data/add.go | 14 ++------ .../internal/data/exphist_test.go | 10 ++++-- .../internal/data/expo/buckets.go | 9 +++-- .../internal/data/expo/expotest/bins.go | 8 +++-- .../internal/data/expo/expotest/equal.go | 8 +++-- .../internal/data/expo/expotest/histogram.go | 8 +++-- .../internal/data/expo/merge.go | 5 ++- .../internal/data/expo/merge_test.go | 3 ++ .../internal/data/expo/scale.go | 5 ++- .../internal/data/expo/scale_test.go | 6 +++- .../internal/data/expo/zero.go | 5 ++- .../internal/data/expo/zero_test.go | 3 ++ .../internal/data/helper_test.go | 34 ++++++------------- 13 files changed, 68 insertions(+), 50 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 5da87608afc7..584d6f4c0ff8 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -6,8 +6,9 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contri import ( "math" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) func (dp Number) Add(in Number) Number { @@ -30,9 +31,7 @@ func (dp Histogram) Add(in Histogram) Histogram { func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { type H = ExpHistogram - - switch { - case dp.Timestamp() >= in.Timestamp(): + if dp.Timestamp() >= in.Timestamp() { panic("out of order") } @@ -79,10 +78,3 @@ type field struct { del func(ExpHistogram) op func(a, b float64) float64 } - -func pos(i int) int { - if i < 0 { - i = -i - } - return i -} diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go index 578616fe9291..2fc4f6662d20 100644 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go @@ -1,15 +1,19 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package data import ( "math" "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" ) // represents none/absent/unset in several tests diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go index a8d02df65b39..83e35a4d778b 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go @@ -1,4 +1,7 @@ -package expo +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( "go.opentelemetry.io/collector/pdata/pcommon" @@ -21,11 +24,11 @@ func (b Buckets) SetAt(i int, v uint64) { } func (b Buckets) Len() int { - return b.data().Len() + int(b.Offset()) + return b.data().Len() + b.Offset() } func (b Buckets) EnsureLen(n int) { - sz := n - int(b.Offset()) + sz := n - b.Offset() b.data().EnsureCapacity(sz) b.data().Append(make([]uint64, sz-b.data().Len())...) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go index 5e9a3a284ddd..725f04b38a08 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -1,11 +1,15 @@ -package expotest +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expotest // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" import ( "fmt" "math" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) const ( diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index 6fa68661dce8..367e15c4e159 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -1,11 +1,15 @@ -package expotest +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expotest // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" import ( "testing" "github.com/matryer/is" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type I struct { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go index 19bb979697a6..1acf79da14c7 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go @@ -1,8 +1,12 @@ -package expotest +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expotest // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" import ( - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type Histogram struct { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index 84e3ac604038..0e6acf88b75e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -1,4 +1,7 @@ -package expo +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( "go.opentelemetry.io/collector/pdata/pcommon" diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go index 06f3855fe343..8e763c682f82 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package expo_test import ( diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index c3ea306e7f73..d98220af1662 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -1,4 +1,7 @@ -package expo +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( "math" diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go index 4cd3ae536aa1..e545e1ee2a41 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package expo_test import ( @@ -7,8 +10,9 @@ import ( "testing" "github.com/matryer/is" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) func TestDownscale(t *testing.T) { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go index 14095e219ec9..cc3dfdb4a068 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -1,4 +1,7 @@ -package expo +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( "fmt" diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go index da2af840c013..613d1a9ec43d 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package expo_test import ( diff --git a/processor/deltatocumulativeprocessor/internal/data/helper_test.go b/processor/deltatocumulativeprocessor/internal/data/helper_test.go index 58b08ed102f4..548d41d8629b 100644 --- a/processor/deltatocumulativeprocessor/internal/data/helper_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/helper_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package data import ( @@ -7,9 +10,10 @@ import ( "strings" "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type bins []uint64 @@ -56,22 +60,6 @@ func (bins bins) String() string { return b.String() } -func from(buckets expo.Buckets) bins { - counts := pmetric.ExponentialHistogramDataPointBuckets(buckets).BucketCounts() - off := buckets.Offset() - if off < 0 { - off = -off - } - bs := make(bins, counts.Len()+off) - for i := 0; i < off; i++ { - bs[i] = ø - } - for i := 0; i < counts.Len(); i++ { - bs[i+off] = counts.At(i) - } - return bs -} - func TestBucketsHelper(t *testing.T) { cases := []struct { bins bins @@ -116,9 +104,9 @@ func zeros(size int) bins { } // observe some points with scale 0 -func (dps bins) observe0(pts ...float64) bins { +func (bins bins) observe0(pts ...float64) bins { if len(pts) == 0 { - return dps + return bins } // https://opentelemetry.io/docs/specs/otel/metrics/data-model/#scale-zero-extract-the-exponent @@ -134,15 +122,15 @@ func (dps bins) observe0(pts ...float64) bins { min := idx(pts[0]) max := idx(pts[len(pts)-1]) for i := min; i <= max; i++ { - if dps[i] == ø { - dps[i] = 0 + if bins[i] == ø { + bins[i] = 0 } } for _, pt := range pts { - dps[idx(pt)] += 1 + bins[idx(pt)]++ } - return dps + return bins } func TestObserve0(t *testing.T) { From 41a2301038cd4ab39f026e6ab08cf73affa4f855 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 22 Apr 2024 17:11:23 +0200 Subject: [PATCH 07/24] data: use expotest converts TestAdd to expotest test helpers. --- .../internal/data/exphist_test.go | 229 ------------------ .../internal/data/expo/buckets.go | 95 -------- .../internal/data/expo/expotest/bins.go | 2 +- .../internal/data/expo/expotest/equal.go | 11 +- .../internal/data/expo/expotest/histogram.go | 45 +++- .../internal/data/expo/merge_test.go | 6 +- .../internal/data/expo/ord.go | 18 ++ .../internal/data/expo/zero_test.go | 20 +- .../internal/data/expo_test.go | 126 ++++++++++ .../internal/data/helper_test.go | 152 ------------ 10 files changed, 202 insertions(+), 502 deletions(-) delete mode 100644 processor/deltatocumulativeprocessor/internal/data/exphist_test.go delete mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/buckets.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/ord.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo_test.go delete mode 100644 processor/deltatocumulativeprocessor/internal/data/helper_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go b/processor/deltatocumulativeprocessor/internal/data/exphist_test.go deleted file mode 100644 index 2fc4f6662d20..000000000000 --- a/processor/deltatocumulativeprocessor/internal/data/exphist_test.go +++ /dev/null @@ -1,229 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package data - -import ( - "math" - "testing" - - "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/pdata/pcommon" - "go.opentelemetry.io/collector/pdata/pmetric" - - "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" -) - -// represents none/absent/unset in several tests -const ø = math.MaxUint64 - -func TestAdd(t *testing.T) { - type tcase struct { - name string - dp, in expdp - want expdp - } - - cases := []tcase{{ - name: "noop", - dp: expdp{ts: 0, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - in: expdp{ts: 1, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - want: expdp{ts: 1, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - }, { - name: "simple", - dp: expdp{ts: 0, pos: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, neg: bins{0, 0, 0, 0, 0, 0, 0, 0, 0}, count: 0}, - in: expdp{ts: 1, pos: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * 45}, - want: expdp{ts: 1, pos: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, neg: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, count: 2 * (0 + 45)}, - }, { - name: "lower+shorter", - dp: expdp{ts: 0, pos: bins{ø, ø, ø, ø, ø, 1, 1, 1, 1}, neg: bins{ø, ø, ø, ø, ø, 1, 1, 1, 1}, count: 2 * 4}, - in: expdp{ts: 1, pos: bins{ø, ø, 1, 1, 1, 1, 1, ø, ø}, neg: bins{ø, ø, 1, 1, 1, 1, 1, ø, ø}, count: 2 * 5}, - want: expdp{ts: 1, pos: bins{ø, ø, 1, 1, 1, 2, 2, 1, 1}, neg: bins{ø, ø, 1, 1, 1, 2, 2, 1, 1}, count: 2 * (4 + 5)}, - }, { - name: "longer", - dp: expdp{ts: 0, pos: bins{1, 1, 1, 1, 1, 1, ø, ø, ø}, neg: bins{1, 1, 1, 1, 1, 1, ø, ø, ø}, count: 2 * 6}, - in: expdp{ts: 1, pos: bins{1, 1, 1, 1, 1, 1, 1, 1, 1}, neg: bins{1, 1, 1, 1, 1, 1, 1, 1, 1}, count: 2 * 9}, - want: expdp{ts: 1, pos: bins{2, 2, 2, 2, 2, 2, 1, 1, 1}, neg: bins{2, 2, 2, 2, 2, 2, 1, 1, 1}, count: 2 * (6 + 9)}, - }, { - name: "optional/missing-dp", - dp: expdp{ts: 0, pos: zeros(5).observe0(0.6, 2.4) /* */, count: 2}, - in: expdp{ts: 1, pos: zeros(5).observe0(1.5, 3.2, 6.3), min: some(1.5), max: some(6.3), sum: some(11.0), count: 3}, - want: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4, 1.5, 3.2, 6.3) /* */, count: 5}, - }, { - name: "optional/missing-in", - dp: expdp{ts: 0, pos: zeros(5).observe0(1.5, 3.2, 6.3), min: some(1.5), max: some(6.3), sum: some(11.0), count: 3}, - in: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4) /* */, count: 2}, - want: expdp{ts: 1, pos: zeros(5).observe0(0.6, 2.4, 1.5, 3.2, 6.3) /* */, count: 5}, - }, { - name: "min-max-sum", - dp: expdp{ts: 0, pos: zeros(5).observe0(1.5, 5.3, 11.6) /* */, min: some(1.5), max: some(11.6), sum: some(18.4), count: 3}, - in: expdp{ts: 1, pos: zeros(5).observe0(0.6, 3.3, 7.9) /* */, min: some(0.6), max: some(07.9), sum: some(11.8), count: 3}, - want: expdp{ts: 1, pos: zeros(5).observe0(1.5, 5.3, 11.6, 0.6, 3.3, 7.9), min: some(0.6), max: some(11.6), sum: some(30.2), count: 6}, - }, { - name: "zero/count", - dp: expdp{ts: 0, pos: bins{1, 2}, zc: u64(3), count: 5}, - in: expdp{ts: 1, pos: bins{1, 0}, zc: u64(2), count: 3}, - want: expdp{ts: 1, pos: bins{2, 2}, zc: u64(5), count: 8}, - }, { - // (1, 2], (2, 4], (4, 8], (8, 16], (16, 32] - name: "zero/diff-simple", - dp: expdp{ts: 0, pos: bins{1, 1, 1}, zt: some(0.0), zc: u64(2)}, - in: expdp{ts: 1, pos: bins{ø, 1, 1}, zt: some(2.0), zc: u64(2)}, - want: expdp{ts: 1, pos: bins{ø, 2, 2}, zt: some(2.0), zc: u64((2 + 2) + 1)}, - }, { - name: "negative-offset", - dp: expdp{ts: 0, posb: some(buckets([]uint64{ /* */ 1, 2}, -2))}, - in: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3 /* */}, -5))}, - want: expdp{ts: 1, posb: some(buckets([]uint64{1, 2, 3, 1, 2}, -5))}, - }, { - name: "scale/diff", - dp: expdp{ts: 0, posb: some(expo.Buckets(expotest.Observe(expo.Scale(1), 1, 2, 3, 4))), scale: 1}, - in: expdp{ts: 1, posb: some(expo.Buckets(expotest.Observe(expo.Scale(0), 1, 2, 3, 4))), scale: 0}, - want: expdp{ts: 1, scale: 0, posb: func() *expo.Buckets { - bs := pmetric.NewExponentialHistogramDataPointBuckets() - expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) - expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) - bs.BucketCounts().Append([]uint64{0, 0}...) // rescaling leaves zeroed memory. this is expected - return some(expo.Buckets(bs)) - }()}, - }} - - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - dp, in := c.dp.into(), c.in.into() - want := c.want.into() - - got := dp.Add(in) - - is := expotest.Is(t) - is.Equal(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint) - - if err := pmetrictest.CompareExponentialHistogramDataPoint(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint); err != nil { - t.Fatal(err) - } - }) - } -} - -func TestBucketsIter(t *testing.T) { - type tcase struct { - name string - data expo.Buckets - want []uint64 - } - - cases := []tcase{{ - name: "full", - data: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}.into(), - want: bins{1, 2, 3, 4, 5, 6, 7, 8, 9}, - }, { - name: "3-6", - data: bins{ø, ø, ø, 4, 5, 6, ø, ø, ø, ø}.into(), - want: bins{0, 0, 0, 4, 5, 6}, - }} - - for _, c := range cases { - t.Run(c.name, func(t *testing.T) { - got := make([]uint64, c.data.Len()) - for i := 0; i < c.data.Len(); i++ { - got[i] = c.data.At(i) - } - require.ElementsMatch(t, c.want, got) - require.Equal(t, len(c.want), len(got)) - }) - } -} - -func TestBucketsSet(t *testing.T) { - var ( - aggr = bins{0, 0, 0, 0, 0, 0, 0, 0, 0}.into() - bkt0 = bins{ø, 1, 1, 1, 1, ø, ø, ø, ø}.into() - bkt1 = bins{ø, ø, ø, 1, 1, 1, 1, ø, ø}.into() - want = bins{0, 1, 1, 2, 2, 1, 1, 0, 0} - ) - - for i := 0; i < aggr.Len(); i++ { - aggr.SetAt(i, bkt0.At(i)+bkt1.At(i)) - } - - got := make(bins, aggr.Len()) - for i := 0; i < aggr.Len(); i++ { - got[i] = aggr.At(i) - } - require.Equal(t, want, got) -} - -func TestBucketsEnsureLen(t *testing.T) { - var ( - data = bins{ø, ø, 3, 4} - want = bins{0, 0, 3, 4, 0, 0, 0} - ) - bkt := data.into() - bkt.EnsureLen(len(want)) - - got := make(bins, bkt.Len()) - for i := 0; i < bkt.Len(); i++ { - got[i] = bkt.At(i) - } - require.Equal(t, want, got) -} - -type expdp struct { - ts int - - pos, neg bins - posb, negb *expo.Buckets - - scale int32 - count uint64 - sum *float64 - - min, max *float64 - - zc *uint64 - zt *float64 -} - -func (e expdp) into() ExpHistogram { - dp := pmetric.NewExponentialHistogramDataPoint() - dp.SetTimestamp(pcommon.Timestamp(e.ts)) - - posb := e.pos.into() - if e.posb != nil { - posb = *e.posb - } - pmetric.ExponentialHistogramDataPointBuckets(posb).CopyTo(dp.Positive()) - - negb := e.neg.into() - if e.negb != nil { - negb = *e.negb - } - pmetric.ExponentialHistogramDataPointBuckets(negb).CopyTo(dp.Negative()) - - dp.SetScale(e.scale) - dp.SetCount(e.count) - - setnn(e.sum, dp.SetSum) - setnn(e.min, dp.SetMin) - setnn(e.max, dp.SetMax) - setnn(e.zc, dp.SetZeroCount) - setnn(e.zt, dp.SetZeroThreshold) - - return ExpHistogram{ExponentialHistogramDataPoint: dp} -} - -func setnn[T any](ptr *T, set func(T)) { - if ptr != nil { - set(*ptr) - } -} - -func some[T any](v T) *T { - return &v -} - -func u64(v uint64) *uint64 { - return &v -} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go b/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go deleted file mode 100644 index 83e35a4d778b..000000000000 --- a/processor/deltatocumulativeprocessor/internal/data/expo/buckets.go +++ /dev/null @@ -1,95 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" - -import ( - "go.opentelemetry.io/collector/pdata/pcommon" - "go.opentelemetry.io/collector/pdata/pmetric" -) - -type Buckets pmetric.ExponentialHistogramDataPointBuckets - -func (b Buckets) At(i int) uint64 { - if idx, ok := b.idx(i); ok { - return b.data().At(idx) - } - return 0 -} - -func (b Buckets) SetAt(i int, v uint64) { - if idx, ok := b.idx(i); ok { - b.data().SetAt(idx, v) - } -} - -func (b Buckets) Len() int { - return b.data().Len() + b.Offset() -} - -func (b Buckets) EnsureLen(n int) { - sz := n - b.Offset() - b.data().EnsureCapacity(sz) - b.data().Append(make([]uint64, sz-b.data().Len())...) -} - -func (b Buckets) Truncate(first int) { - offset := first - b.Offset() - if b.Offset() >= offset { - return - } - - data := b.data().AsRaw()[offset-b.Offset():] - b.data().FromRaw(data) - b.as().SetOffset(int32(offset)) -} - -// Expand the buckets by n slots: -// - n < 0: prepend at front, lowering offset -// - n > 0: append to back -func (b Buckets) Expand(n int) { - switch { - case n < 0: - n = -n - us := pcommon.NewUInt64Slice() - us.Append(make([]uint64, n+b.data().Len())...) - for i := 0; i < b.data().Len(); i++ { - us.SetAt(i+n, b.data().At(i)) - } - us.MoveTo(b.data()) - b.as().SetOffset(int32(b.Offset() - n)) - case n > 0: - b.data().Append(make([]uint64, n)...) - } -} - -func (b Buckets) Offset() int { - return int(b.as().Offset()) -} - -func (b Buckets) idx(i int) (int, bool) { - idx := i - b.Offset() - return idx, idx >= 0 && idx < b.data().Len() -} - -func (b Buckets) data() pcommon.UInt64Slice { - return b.as().BucketCounts() -} - -func (b Buckets) as() pmetric.ExponentialHistogramDataPointBuckets { - return pmetric.ExponentialHistogramDataPointBuckets(b) -} - -type ord interface { - int | int32 | float64 -} - -// HiLo returns the greater of a and b by comparing the result of applying fn to -// each -func HiLo[T any, N ord](a, b T, fn func(T) N) (hi, lo T) { - an, bn := fn(a), fn(b) - if an > bn { - return a, b - } - return b, a -} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go index 725f04b38a08..73a91163c48d 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -22,7 +22,7 @@ const ( // bounds: (0.125,0.25], (0.25,0.5], (0.5,1], (1,2], (2,4], (4,8], (8,16], (16,32] type Bins [8]uint64 -func Buckets(bins Bins) pmetric.ExponentialHistogramDataPointBuckets { +func (bins Bins) Into() pmetric.ExponentialHistogramDataPointBuckets { start := 0 for i := 0; i < len(bins); i++ { if bins[i] != ø { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index 367e15c4e159..f2ec55001f4b 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -9,6 +9,7 @@ import ( "github.com/matryer/is" "go.opentelemetry.io/collector/pdata/pmetric" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) @@ -24,10 +25,16 @@ func (is *I) Equal(want, got any) { switch got := got.(type) { case expo.DataPoint: want := want.(expo.DataPoint) - is.I.Equal(want.ZeroCount(), got.ZeroCount()) // zero-count - is.I.Equal(want.ZeroThreshold(), got.ZeroThreshold()) // zero-threshold + is.I.Equal(want.Timestamp(), got.Timestamp()) // time + is.I.Equal(want.StartTimestamp(), got.StartTimestamp()) // start-time + is.I.Equal(want.Scale(), got.Scale()) // scale + is.I.Equal(want.Count(), got.Count()) // count + is.I.Equal(want.Sum(), got.Sum()) // sum + is.I.Equal(want.ZeroCount(), got.ZeroCount()) // zero-count + is.I.Equal(want.ZeroThreshold(), got.ZeroThreshold()) // zero-threshold is.Equal(want.Positive(), got.Positive()) is.Equal(want.Negative(), got.Negative()) + is.NoErr(pmetrictest.CompareExponentialHistogramDataPoint(want, got)) // pmetrictest case pmetric.ExponentialHistogramDataPointBuckets: want := want.(pmetric.ExponentialHistogramDataPointBuckets) is.I.Equal(want.Offset(), got.Offset()) // offset diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go index 1acf79da14c7..98c868b9c97e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go @@ -4,33 +4,54 @@ package expotest // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" import ( + "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type Histogram struct { - Pos, Neg Bins - PosNeg Bins + Ts pcommon.Timestamp + + Pos, Neg pmetric.ExponentialHistogramDataPointBuckets + PosNeg pmetric.ExponentialHistogramDataPointBuckets Scale int + Count uint64 + Sum *float64 + + Min, Max *float64 Zt float64 Zc uint64 } -func DataPoint(hist Histogram) expo.DataPoint { +func (hist Histogram) Into() expo.DataPoint { dp := pmetric.NewExponentialHistogramDataPoint() + dp.SetTimestamp(hist.Ts) + + if !zero(hist.PosNeg) { + hist.PosNeg.CopyTo(dp.Positive()) + hist.PosNeg.CopyTo(dp.Negative()) + } + + if !zero(hist.Pos) { + hist.Pos.MoveTo(dp.Positive()) + } + if !zero(hist.Neg) { + hist.Neg.MoveTo(dp.Negative()) + } - posneg := Buckets(hist.PosNeg) - posneg.CopyTo(dp.Positive()) - posneg.CopyTo(dp.Negative()) + dp.SetCount(hist.Count) + if hist.Sum != nil { + dp.SetSum(*hist.Sum) + } - if (hist.Pos != Bins{}) { - Buckets(hist.Pos).MoveTo(dp.Positive()) + if hist.Min != nil { + dp.SetMin(*hist.Min) } - if (hist.Neg != Bins{}) { - Buckets(hist.Neg).MoveTo(dp.Negative()) + if hist.Max != nil { + dp.SetMax(*hist.Max) } dp.SetScale(int32(hist.Scale)) @@ -38,3 +59,7 @@ func DataPoint(hist Histogram) expo.DataPoint { dp.SetZeroCount(hist.Zc) return dp } + +func zero[T comparable](v T) bool { + return v == *new(T) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go index 8e763c682f82..4d3791721bcd 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge_test.go @@ -39,9 +39,9 @@ func TestMerge(t *testing.T) { }} for _, cs := range cases { - a := expotest.Buckets(cs.a) - b := expotest.Buckets(cs.b) - want := expotest.Buckets(cs.want) + a := cs.a.Into() + b := cs.b.Into() + want := cs.want.Into() name := fmt.Sprintf("(%+d,%d)+(%+d,%d)=(%+d,%d)", a.Offset(), a.BucketCounts().Len(), b.Offset(), b.BucketCounts().Len(), want.Offset(), want.BucketCounts().Len()) t.Run(name, func(t *testing.T) { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/ord.go b/processor/deltatocumulativeprocessor/internal/data/expo/ord.go new file mode 100644 index 000000000000..af5527b44244 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/ord.go @@ -0,0 +1,18 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + +type ord interface { + int | int32 | float64 +} + +// HiLo returns the greater of a and b by comparing the result of applying fn to +// each +func HiLo[T any, N ord](a, b T, fn func(T) N) (hi, lo T) { + an, bn := fn(a), fn(b) + if an > bn { + return a, b + } + return b, a +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go index 613d1a9ec43d..4e0e3e56d029 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -24,25 +24,25 @@ func TestWidenZero(t *testing.T) { // (0.125,0.25], (0.25,0.5], (0.5,1], (1,2], (2,4], (4,8], (8,16], (16,32] // // -3 -2 -1 0 1 2 3 4 - hist: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}, Zt: 0, Zc: 0}, - want: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}, Zt: 0, Zc: 0}, + hist: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}.Into(), Zt: 0, Zc: 0}, + want: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, ø, ø}.Into(), Zt: 0, Zc: 0}, }, { // zt=2 is upper boundary of bucket 0. keep buckets [1:n] - hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}, Zt: 0, Zc: 2}, - want: hist{PosNeg: bins{ø, ø, ø, ø, 3, 4, 5, ø}, Zt: 2, Zc: 2 + 2*(1+2)}, + hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}.Into(), Zt: 0, Zc: 2}, + want: hist{PosNeg: bins{ø, ø, ø, ø, 3, 4, 5, ø}.Into(), Zt: 2, Zc: 2 + 2*(1+2)}, }, { // zt=3 is within bucket 1. keep buckets [2:n] // set zt=4 because it must cover full buckets - hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}, Zt: 0, Zc: 2}, + hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}.Into(), Zt: 0, Zc: 2}, min: 3, - want: hist{PosNeg: bins{ø, ø, ø, ø, ø, 4, 5, ø}, Zt: 4, Zc: 2 + 2*(1+2+3)}, + want: hist{PosNeg: bins{ø, ø, ø, ø, ø, 4, 5, ø}.Into(), Zt: 4, Zc: 2 + 2*(1+2+3)}, }} for _, cs := range cases { name := fmt.Sprintf("%.2f->%.2f", cs.hist.Zt, cs.want.Zt) t.Run(name, func(t *testing.T) { - hist := expotest.DataPoint(cs.hist) - want := expotest.DataPoint(cs.want) + hist := cs.hist.Into() + want := cs.want.Into() zt := cs.min if zt == 0 { @@ -90,8 +90,8 @@ func TestSlice(t *testing.T) { to -= 3 t.Run(fmt.Sprintf("[%d:%d]", from, to), func(t *testing.T) { - bins := expotest.Buckets(cs.bins) - want := expotest.Buckets(cs.want) + bins := cs.bins.Into() + want := cs.want.Into() expo.Abs(bins).Slice(from, to) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo_test.go new file mode 100644 index 000000000000..eab68e3633fc --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo_test.go @@ -0,0 +1,126 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package data + +import ( + "math" + "testing" + + "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" +) + +// represents none/absent/unset in several tests +const ø = math.MaxUint64 + +func TestAdd(t *testing.T) { + type expdp = expotest.Histogram + type bins = expotest.Bins + var obs0 = expotest.Observe0 + + cases := []struct { + name string + dp, in expdp + want expdp + flip bool + }{{ + name: "noop", + dp: expdp{PosNeg: bins{0, 0, 0, 0, 0, 0, 0, 0}.Into(), Count: 0}, + in: expdp{PosNeg: bins{0, 0, 0, 0, 0, 0, 0, 0}.Into(), Count: 0}, + want: expdp{PosNeg: bins{0, 0, 0, 0, 0, 0, 0, 0}.Into(), Count: 0}, + }, { + name: "simple", + dp: expdp{PosNeg: bins{0, 0, 0, 0, 0, 0, 0, 0}.Into(), Count: 0}, + in: expdp{PosNeg: bins{1, 2, 3, 4, 5, 6, 7, 8}.Into(), Count: 2 * (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8)}, + want: expdp{PosNeg: bins{1, 2, 3, 4, 5, 6, 7, 8}.Into(), Count: 2 * (0 + (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8))}, + }, { + name: "lower+shorter", + dp: expdp{PosNeg: bins{ø, ø, ø, ø, ø, 1, 1, 1}.Into(), Count: 2 * 3}, + in: expdp{PosNeg: bins{ø, ø, 1, 1, 1, 1, 1, ø}.Into(), Count: 2 * 5}, + want: expdp{PosNeg: bins{ø, ø, 1, 1, 1, 2, 2, 1}.Into(), Count: 2 * (3 + 5)}, + }, { + name: "longer", + dp: expdp{PosNeg: bins{1, 1, 1, 1, 1, ø, ø, ø}.Into(), Count: 2 * 5}, + in: expdp{PosNeg: bins{1, 1, 1, 1, 1, 1, 1, 1}.Into(), Count: 2 * 8}, + want: expdp{PosNeg: bins{2, 2, 2, 2, 2, 1, 1, 1}.Into(), Count: 2 * (5 + 8)}, + }, { + name: "optional/missing", flip: true, + dp: expdp{PosNeg: obs0(0.6, 2.4) /* */, Count: 2}, + in: expdp{PosNeg: obs0(1.5, 3.2, 6.3), Min: some(1.5), Max: some(6.3), Sum: some(11.0), Count: 3}, + want: expdp{PosNeg: obs0(0.6, 2.4, 1.5, 3.2, 6.3) /* */, Count: 5}, + }, { + name: "optional/min-max-sum", + dp: expdp{PosNeg: obs0(1.5, 5.3, 11.6) /* */, Min: some(1.5), Max: some(11.6), Sum: some(18.4), Count: 3}, + in: expdp{PosNeg: obs0(0.6, 3.3, 7.9) /* */, Min: some(0.6), Max: some(07.9), Sum: some(11.8), Count: 3}, + want: expdp{PosNeg: obs0(1.5, 5.3, 11.6, 0.6, 3.3, 7.9), Min: some(0.6), Max: some(11.6), Sum: some(30.2), Count: 6}, + }, { + name: "zero/count", + dp: expdp{PosNeg: bins{0, 1, 2}.Into(), Zt: 0, Zc: 3, Count: 5}, + in: expdp{PosNeg: bins{0, 1, 0}.Into(), Zt: 0, Zc: 2, Count: 3}, + want: expdp{PosNeg: bins{0, 2, 2}.Into(), Zt: 0, Zc: 5, Count: 8}, + }, { + name: "zero/diff", + dp: expdp{Pos: bins{ø, ø, 0, 1, 1, 1}.Into(), Zt: 0.0, Zc: 2}, + in: expdp{Pos: bins{ø, ø, ø, ø, 1, 1}.Into(), Zt: 2.0, Zc: 2}, + want: expdp{Pos: bins{ø, ø, ø, ø, 2, 2}.Into(), Zt: 2.0, Zc: 2 + 2 + 1}, + }, { + name: "negative-offset", + dp: expdp{PosNeg: rawbs([]uint64{ /* */ 1, 2}, -2)}, + in: expdp{PosNeg: rawbs([]uint64{1, 2, 3 /* */}, -5)}, + want: expdp{PosNeg: rawbs([]uint64{1, 2, 3, 1, 2}, -5)}, + }, { + name: "scale/diff", + dp: expdp{PosNeg: expotest.Observe(expo.Scale(1), 1, 2, 3, 4), Scale: 1}, + in: expdp{PosNeg: expotest.Observe(expo.Scale(0), 4, 3, 2, 1), Scale: 0}, + want: expdp{Scale: 0, PosNeg: func() pmetric.ExponentialHistogramDataPointBuckets { + bs := pmetric.NewExponentialHistogramDataPointBuckets() + expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) + expotest.ObserveInto(bs, expo.Scale(0), 4, 3, 2, 1) + bs.BucketCounts().Append([]uint64{0, 0}...) // rescaling leaves zeroed memory. this is expected + return bs + }()}, + }} + + for _, cs := range cases { + run := func(dp, in expdp) func(t *testing.T) { + return func(t *testing.T) { + is := expotest.Is(t) + + var ( + dp = ExpHistogram{dp.Into()} + in = ExpHistogram{in.Into()} + want = ExpHistogram{cs.want.Into()} + ) + + dp.SetTimestamp(0) + in.SetTimestamp(1) + want.SetTimestamp(1) + + got := dp.Add(in) + is.Equal(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint) + } + } + + if cs.flip { + t.Run(cs.name+"-dp", run(cs.dp, cs.in)) + t.Run(cs.name+"-in", run(cs.in, cs.dp)) + continue + } + t.Run(cs.name, run(cs.dp, cs.in)) + } + +} + +func rawbs(data []uint64, offset int32) pmetric.ExponentialHistogramDataPointBuckets { + bs := pmetric.NewExponentialHistogramDataPointBuckets() + bs.BucketCounts().FromRaw(data) + bs.SetOffset(offset) + return bs +} + +func some[T any](v T) *T { + return &v +} diff --git a/processor/deltatocumulativeprocessor/internal/data/helper_test.go b/processor/deltatocumulativeprocessor/internal/data/helper_test.go deleted file mode 100644 index 548d41d8629b..000000000000 --- a/processor/deltatocumulativeprocessor/internal/data/helper_test.go +++ /dev/null @@ -1,152 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package data - -import ( - "math" - "sort" - "strconv" - "strings" - "testing" - - "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/pdata/pmetric" - - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" -) - -type bins []uint64 - -func (bins bins) into() expo.Buckets { - off, counts := bins.split() - return buckets(counts, offset(len(off))) -} - -func (bins bins) split() (offset, counts []uint64) { - start := 0 - for i := 0; i < len(bins); i++ { - if bins[i] != ø { - start = i - break - } - } - - end := len(bins) - for i := start; i < len(bins); i++ { - if bins[i] == ø { - end = i - break - } - } - - return bins[0:start], bins[start:end] -} - -func (bins bins) String() string { - var b strings.Builder - b.WriteString("[") - for i, v := range bins { - if i != 0 { - b.WriteString(", ") - } - if v == ø { - b.WriteString("_") - continue - } - b.WriteString(strconv.FormatUint(v, 10)) - } - b.WriteString("]") - return b.String() -} - -func TestBucketsHelper(t *testing.T) { - cases := []struct { - bins bins - want expo.Buckets - }{{ - bins: bins{}, - want: buckets(nil, offset(0)), - }, { - bins: bins{1, 2, 3, 4}, - want: buckets([]uint64{1, 2, 3, 4}, offset(0)), - }, { - bins: bins{ø, ø, 3, 4}, - want: buckets([]uint64{3, 4}, offset(2)), - }, { - bins: bins{ø, ø, 3, 4, ø, ø}, - want: buckets([]uint64{3, 4}, offset(2)), - }, { - bins: bins{1, 2, ø, ø}, - want: buckets([]uint64{1, 2}, offset(0)), - }} - - for _, c := range cases { - got := c.bins.into() - require.Equal(t, c.want, got) - } -} - -type offset int - -func buckets(data []uint64, offset offset) expo.Buckets { - if data == nil { - data = make([]uint64, 0) - } - bs := pmetric.NewExponentialHistogramDataPointBuckets() - bs.BucketCounts().FromRaw(data) - bs.SetOffset(int32(offset)) - return expo.Buckets(bs) -} - -func zeros(size int) bins { - return make(bins, size) -} - -// observe some points with scale 0 -func (bins bins) observe0(pts ...float64) bins { - if len(pts) == 0 { - return bins - } - - // https://opentelemetry.io/docs/specs/otel/metrics/data-model/#scale-zero-extract-the-exponent - idx := func(v float64) int { - frac, exp := math.Frexp(v) - if frac == 0.5 { - exp-- - } - return exp - } - - sort.Float64s(pts) - min := idx(pts[0]) - max := idx(pts[len(pts)-1]) - for i := min; i <= max; i++ { - if bins[i] == ø { - bins[i] = 0 - } - } - - for _, pt := range pts { - bins[idx(pt)]++ - } - return bins -} - -func TestObserve0(t *testing.T) { - cases := []struct { - pts []float64 - want bins - }{{ - pts: []float64{1.5, 5.3, 11.6}, - want: bins{0, 1, 0, 1, 1}, - }, { - pts: []float64{0.6, 3.3, 7.9}, - want: bins{1, 0, 1, 1, 0}, - }} - - for _, c := range cases { - got := zeros(len(c.want)).observe0(c.pts...) - require.Equal(t, c.want, got) - } -} From aae577d2aee9bd3e0d0029ad1af1bb4826ad174f Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 22 Apr 2024 17:22:42 +0200 Subject: [PATCH 08/24] expo: type aliases --- .../deltatocumulativeprocessor/internal/data/add.go | 2 +- .../deltatocumulativeprocessor/internal/data/data.go | 9 +++++---- .../internal/data/expo/expo.go | 8 ++++++++ .../internal/data/expo/expotest/bins.go | 8 ++++---- .../internal/data/expo/expotest/equal.go | 5 ++--- .../internal/data/expo/expotest/histogram.go | 4 ++-- .../internal/data/expo/merge.go | 5 ++--- .../internal/data/expo/scale.go | 6 ++---- .../internal/data/expo/scale_test.go | 2 +- .../internal/data/expo/zero.go | 6 +----- .../internal/data/expo_test.go | 6 +++--- .../deltatocumulativeprocessor/internal/metrics/data.go | 2 +- 12 files changed, 32 insertions(+), 31 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expo.go diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 584d6f4c0ff8..75955f92dba1 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -45,7 +45,7 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { if dp.ZeroThreshold() != in.ZeroThreshold() { hi, lo := expo.HiLo(dp, in, ExpHistogram.ZeroThreshold) - expo.WidenZero(lo.ExponentialHistogramDataPoint, hi.ZeroThreshold()) + expo.WidenZero(lo.DataPoint, hi.ZeroThreshold()) } expo.Merge(dp.Positive(), in.Positive()) diff --git a/processor/deltatocumulativeprocessor/internal/data/data.go b/processor/deltatocumulativeprocessor/internal/data/data.go index 941b3cff904f..0e5254e02034 100644 --- a/processor/deltatocumulativeprocessor/internal/data/data.go +++ b/processor/deltatocumulativeprocessor/internal/data/data.go @@ -4,6 +4,7 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" import ( + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" ) @@ -52,19 +53,19 @@ func (dp Histogram) CopyTo(dst Histogram) { } type ExpHistogram struct { - pmetric.ExponentialHistogramDataPoint + expo.DataPoint } func (dp ExpHistogram) Clone() ExpHistogram { - clone := ExpHistogram{ExponentialHistogramDataPoint: pmetric.NewExponentialHistogramDataPoint()} - if dp.ExponentialHistogramDataPoint != (pmetric.ExponentialHistogramDataPoint{}) { + clone := ExpHistogram{DataPoint: pmetric.NewExponentialHistogramDataPoint()} + if dp.DataPoint != (expo.DataPoint{}) { dp.CopyTo(clone) } return clone } func (dp ExpHistogram) CopyTo(dst ExpHistogram) { - dp.ExponentialHistogramDataPoint.CopyTo(dst.ExponentialHistogramDataPoint) + dp.DataPoint.CopyTo(dst.DataPoint) } type mustPoint[D Point[D]] struct{ _ D } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go new file mode 100644 index 000000000000..22c3a8f6377c --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go @@ -0,0 +1,8 @@ +package expo + +import "go.opentelemetry.io/collector/pdata/pmetric" + +type ( + DataPoint = pmetric.ExponentialHistogramDataPoint + Buckets = pmetric.ExponentialHistogramDataPointBuckets +) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go index 73a91163c48d..13b4ce74c928 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/bins.go @@ -22,7 +22,7 @@ const ( // bounds: (0.125,0.25], (0.25,0.5], (0.5,1], (1,2], (2,4], (4,8], (8,16], (16,32] type Bins [8]uint64 -func (bins Bins) Into() pmetric.ExponentialHistogramDataPointBuckets { +func (bins Bins) Into() expo.Buckets { start := 0 for i := 0; i < len(bins); i++ { if bins[i] != ø { @@ -47,7 +47,7 @@ func (bins Bins) Into() pmetric.ExponentialHistogramDataPointBuckets { return buckets } -func ObserveInto(bs pmetric.ExponentialHistogramDataPointBuckets, scale expo.Scale, pts ...float64) { +func ObserveInto(bs expo.Buckets, scale expo.Scale, pts ...float64) { counts := bs.BucketCounts() for _, pt := range pts { @@ -70,12 +70,12 @@ func ObserveInto(bs pmetric.ExponentialHistogramDataPointBuckets, scale expo.Sca } } -func Observe(scale expo.Scale, pts ...float64) pmetric.ExponentialHistogramDataPointBuckets { +func Observe(scale expo.Scale, pts ...float64) expo.Buckets { bs := pmetric.NewExponentialHistogramDataPointBuckets() ObserveInto(bs, scale, pts...) return bs } -func Observe0(pts ...float64) pmetric.ExponentialHistogramDataPointBuckets { +func Observe0(pts ...float64) expo.Buckets { return Observe(0, pts...) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index f2ec55001f4b..3c6bf86b66c2 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -7,7 +7,6 @@ import ( "testing" "github.com/matryer/is" - "go.opentelemetry.io/collector/pdata/pmetric" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" @@ -35,8 +34,8 @@ func (is *I) Equal(want, got any) { is.Equal(want.Positive(), got.Positive()) is.Equal(want.Negative(), got.Negative()) is.NoErr(pmetrictest.CompareExponentialHistogramDataPoint(want, got)) // pmetrictest - case pmetric.ExponentialHistogramDataPointBuckets: - want := want.(pmetric.ExponentialHistogramDataPointBuckets) + case expo.Buckets: + want := want.(expo.Buckets) is.I.Equal(want.Offset(), got.Offset()) // offset is.I.Equal(want.BucketCounts().AsRaw(), got.BucketCounts().AsRaw()) // counts default: diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go index 98c868b9c97e..141dad724d82 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/histogram.go @@ -13,8 +13,8 @@ import ( type Histogram struct { Ts pcommon.Timestamp - Pos, Neg pmetric.ExponentialHistogramDataPointBuckets - PosNeg pmetric.ExponentialHistogramDataPointBuckets + Pos, Neg expo.Buckets + PosNeg expo.Buckets Scale int Count uint64 diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index 0e6acf88b75e..18ce76fe8913 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -5,12 +5,11 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contri import ( "go.opentelemetry.io/collector/pdata/pcommon" - "go.opentelemetry.io/collector/pdata/pmetric" ) // Merge combines the counts of buckets a and b into a. // Both buckets MUST be of same scale -func Merge(arel, brel pmetric.ExponentialHistogramDataPointBuckets) { +func Merge(arel, brel Buckets) { a, b := Abs(arel), Abs(brel) lo := min(a.Lower(), b.Lower()) @@ -33,7 +32,7 @@ func Abs(buckets buckets) Absolute { return Absolute{buckets: buckets} } -type buckets = pmetric.ExponentialHistogramDataPointBuckets +type buckets = Buckets // Absolute addresses bucket counts using an absolute scale, such that the // following holds true: diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index d98220af1662..69f7747ad1c0 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -5,8 +5,6 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contri import ( "math" - - "go.opentelemetry.io/collector/pdata/pmetric" ) type Scale int32 @@ -41,7 +39,7 @@ func (scale Scale) Bounds(index int) (min, max float64) { } // Downscale collapses the buckets of bs until scale 'to' is reached -func Downscale(bs pmetric.ExponentialHistogramDataPointBuckets, from, to Scale) { +func Downscale(bs Buckets, from, to Scale) { switch { case from == to: return @@ -58,7 +56,7 @@ func Downscale(bs pmetric.ExponentialHistogramDataPointBuckets, from, to Scale) // // before: 1 1 1 1 1 1 1 1 1 1 1 1 // after: 2 2 2 2 2 2 0 0 0 0 0 0 -func Collapse(bs pmetric.ExponentialHistogramDataPointBuckets) { +func Collapse(bs Buckets) { counts := bs.BucketCounts() size := counts.Len() / 2 if counts.Len()%2 != 0 { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go index e545e1ee2a41..83d9b9f4ce9b 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -46,7 +46,7 @@ func TestDownscale(t *testing.T) { {scale: 0, bkt: " 4 4 4 "}, }} - type B = pmetric.ExponentialHistogramDataPointBuckets + type B = expo.Buckets for i, reprs := range cases { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { buckets := make([]Repr[B], len(reprs)) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go index cc3dfdb4a068..37e27420e18e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -5,12 +5,8 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contri import ( "fmt" - - "go.opentelemetry.io/collector/pdata/pmetric" ) -type DataPoint = pmetric.ExponentialHistogramDataPoint - // WidenZero widens the zero-bucket to span at least [-width,width], possibly wider // if min falls in the middle of a bucket. // @@ -26,7 +22,7 @@ func WidenZero(dp DataPoint, width float64) { scale := Scale(dp.Scale()) lo := scale.Idx(width) - widen := func(bs pmetric.ExponentialHistogramDataPointBuckets) { + widen := func(bs Buckets) { abs := Abs(bs) for i := abs.Lower(); i <= lo; i++ { dp.SetZeroCount(dp.ZeroCount() + abs.Abs(i)) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo_test.go index eab68e3633fc..ad7b4ed359ee 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo_test.go @@ -75,7 +75,7 @@ func TestAdd(t *testing.T) { name: "scale/diff", dp: expdp{PosNeg: expotest.Observe(expo.Scale(1), 1, 2, 3, 4), Scale: 1}, in: expdp{PosNeg: expotest.Observe(expo.Scale(0), 4, 3, 2, 1), Scale: 0}, - want: expdp{Scale: 0, PosNeg: func() pmetric.ExponentialHistogramDataPointBuckets { + want: expdp{Scale: 0, PosNeg: func() expo.Buckets { bs := pmetric.NewExponentialHistogramDataPointBuckets() expotest.ObserveInto(bs, expo.Scale(0), 1, 2, 3, 4) expotest.ObserveInto(bs, expo.Scale(0), 4, 3, 2, 1) @@ -100,7 +100,7 @@ func TestAdd(t *testing.T) { want.SetTimestamp(1) got := dp.Add(in) - is.Equal(want.ExponentialHistogramDataPoint, got.ExponentialHistogramDataPoint) + is.Equal(want.DataPoint, got.DataPoint) } } @@ -114,7 +114,7 @@ func TestAdd(t *testing.T) { } -func rawbs(data []uint64, offset int32) pmetric.ExponentialHistogramDataPointBuckets { +func rawbs(data []uint64, offset int32) expo.Buckets { bs := pmetric.NewExponentialHistogramDataPointBuckets() bs.BucketCounts().FromRaw(data) bs.SetOffset(offset) diff --git a/processor/deltatocumulativeprocessor/internal/metrics/data.go b/processor/deltatocumulativeprocessor/internal/metrics/data.go index c305c85d781e..f063475055f7 100644 --- a/processor/deltatocumulativeprocessor/internal/metrics/data.go +++ b/processor/deltatocumulativeprocessor/internal/metrics/data.go @@ -47,7 +47,7 @@ type ExpHistogram Metric func (s ExpHistogram) At(i int) data.ExpHistogram { dp := Metric(s).ExponentialHistogram().DataPoints().At(i) - return data.ExpHistogram{ExponentialHistogramDataPoint: dp} + return data.ExpHistogram{DataPoint: dp} } func (s ExpHistogram) Len() int { From 885e39a9b12b9a82652f4786923b9bbda9e1871f Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 22 Apr 2024 17:51:45 +0200 Subject: [PATCH 09/24] expo: TestAbsolute --- .../internal/data/expo/expo.go | 37 +++++++++++++++ .../internal/data/expo/expo_test.go | 47 +++++++++++++++++++ .../internal/data/expo/merge.go | 41 ---------------- 3 files changed, 84 insertions(+), 41 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go index 22c3a8f6377c..7e9ce72c7c9a 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go @@ -6,3 +6,40 @@ type ( DataPoint = pmetric.ExponentialHistogramDataPoint Buckets = pmetric.ExponentialHistogramDataPointBuckets ) + +func Abs(buckets buckets) Absolute { + return Absolute{buckets: buckets} +} + +type buckets = Buckets + +// Absolute addresses bucket counts using an absolute scale, such that it is +// interoperable with [Scale]. +// +// It spans from [[Absolute.Lower]:[Absolute.Upper]] +type Absolute struct { + buckets +} + +// Abs returns the value at absolute index 'at' +func (a Absolute) Abs(at int) uint64 { + if i, ok := a.idx(at); ok { + return a.BucketCounts().At(i) + } + return 0 +} + +// Upper returns the minimal index outside the set, such that every i < Upper +func (a Absolute) Upper() int { + return a.BucketCounts().Len() + int(a.Offset()) +} + +// Lower returns the minimal index inside the set, such that every i >= Lower +func (a Absolute) Lower() int { + return int(a.Offset()) +} + +func (a Absolute) idx(at int) (int, bool) { + idx := at - a.Lower() + return idx, idx >= 0 && idx < a.BucketCounts().Len() +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go new file mode 100644 index 000000000000..65e4368ff190 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go @@ -0,0 +1,47 @@ +package expo_test + +import ( + "fmt" + "testing" + + "github.com/matryer/is" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" +) + +func TestAbsolute(t *testing.T) { + is := is.NewRelaxed(t) + + bs := expotest.Bins{ø, 1, 2, 3, 4, 5, ø, ø}.Into() + abs := expo.Abs(bs) + + lo, up := abs.Lower(), abs.Upper() + is.Equal(-2, lo) // lower bound + is.Equal(3, up) // upper bound + + for i := lo; i < up; i++ { + got := abs.Abs(i) + is.Equal(bs.BucketCounts().At(i+2), got) + } +} + +func ExampleAbsolute() { + nums := []float64{0.4, 2.3, 2.4, 4.5} + + bs := expotest.Observe0(nums...) + abs := expo.Abs(bs) + + fmt.Printf("spans from %d:%d\n", abs.Lower(), abs.Upper()) + + s := expo.Scale(0) + for _, n := range nums { + fmt.Printf("%.1f belongs to bucket %2d: %d\n", n, s.Idx(n), abs.Abs(s.Idx(n))) + } + + // Output: + // spans from -2:3 + // 0.4 belongs to bucket -2: 1 + // 2.3 belongs to bucket 1: 2 + // 2.4 belongs to bucket 1: 2 + // 4.5 belongs to bucket 2: 1 +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index 18ce76fe8913..04c7d3a90b4e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -27,44 +27,3 @@ func Merge(arel, brel Buckets) { a.SetOffset(int32(lo)) counts.MoveTo(a.BucketCounts()) } - -func Abs(buckets buckets) Absolute { - return Absolute{buckets: buckets} -} - -type buckets = Buckets - -// Absolute addresses bucket counts using an absolute scale, such that the -// following holds true: -// -// for i := range counts: Scale(…).Idx(counts[i]) == i -// -// It spans from [[Absolute.Lower]:[Absolute.Upper]] -type Absolute struct { - buckets -} - -// Abs returns the value at absolute index 'at'. The following holds true: -// -// Scale(…).Idx(At(i)) == i -func (a Absolute) Abs(at int) uint64 { - if i, ok := a.idx(at); ok { - return a.BucketCounts().At(i) - } - return 0 -} - -// Upper returns the minimal index outside the set, such that every i < Upper -func (a Absolute) Upper() int { - return a.BucketCounts().Len() + int(a.Offset()) -} - -// Lower returns the minimal index inside the set, such that every i >= Lower -func (a Absolute) Lower() int { - return int(a.Offset()) -} - -func (a Absolute) idx(at int) (int, bool) { - idx := at - a.Lower() - return idx, idx >= 0 && idx < a.BucketCounts().Len() -} From 4babe1ded909ea08b2395d0f4d7b8e123d79b0b6 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 22 Apr 2024 18:10:30 +0200 Subject: [PATCH 10/24] processor: aggregate exponential histograms enables the aggregatoin that was built in the previous commits on the processor --- .../internal/telemetry/metrics.go | 15 +++--- .../deltatocumulativeprocessor/processor.go | 53 +++++++++++++------ 2 files changed, 46 insertions(+), 22 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/telemetry/metrics.go b/processor/deltatocumulativeprocessor/internal/telemetry/metrics.go index f3b88ef8b96a..946ffd98d1d6 100644 --- a/processor/deltatocumulativeprocessor/internal/telemetry/metrics.go +++ b/processor/deltatocumulativeprocessor/internal/telemetry/metrics.go @@ -19,11 +19,14 @@ import ( type Telemetry struct { Metrics + + meter metric.Meter } func New(meter metric.Meter) Telemetry { return Telemetry{ Metrics: metrics(meter), + meter: meter, } } @@ -89,23 +92,23 @@ func metrics(meter metric.Meter) Metrics { } } -func (m Metrics) WithLimit(meter metric.Meter, max int64) { +func (tel Telemetry) WithLimit(max int64) { then := metric.Callback(func(_ context.Context, o metric.Observer) error { - o.ObserveInt64(m.streams.limit, max) + o.ObserveInt64(tel.streams.limit, max) return nil }) - _, err := meter.RegisterCallback(then, m.streams.limit) + _, err := tel.meter.RegisterCallback(then, tel.streams.limit) if err != nil { panic(err) } } -func (m Metrics) WithStale(meter metric.Meter, max time.Duration) { +func (tel Telemetry) WithStale(max time.Duration) { then := metric.Callback(func(_ context.Context, o metric.Observer) error { - o.ObserveInt64(m.streams.stale, int64(max.Seconds())) + o.ObserveInt64(tel.streams.stale, int64(max.Seconds())) return nil }) - _, err := meter.RegisterCallback(then, m.streams.stale) + _, err := tel.meter.RegisterCallback(then, tel.streams.stale) if err != nil { panic(err) } diff --git a/processor/deltatocumulativeprocessor/processor.go b/processor/deltatocumulativeprocessor/processor.go index 59fe2c7c4c0c..de4680fb66e8 100644 --- a/processor/deltatocumulativeprocessor/processor.go +++ b/processor/deltatocumulativeprocessor/processor.go @@ -34,8 +34,8 @@ type Processor struct { ctx context.Context cancel context.CancelFunc - aggr streams.Aggregator[data.Number] - stale maybe.Ptr[staleness.Staleness[data.Number]] + sums Pipeline[data.Number] + expo Pipeline[data.ExpHistogram] mtx sync.Mutex } @@ -43,29 +43,43 @@ type Processor struct { func newProcessor(cfg *Config, log *zap.Logger, meter metric.Meter, next consumer.Metrics) *Processor { ctx, cancel := context.WithCancel(context.Background()) + tel := telemetry.New(meter) + proc := Processor{ log: log, ctx: ctx, cancel: cancel, next: next, + + sums: pipeline[data.Number](cfg, &tel), + expo: pipeline[data.ExpHistogram](cfg, &tel), } - tel := telemetry.New(meter) + return &proc +} - var dps streams.Map[data.Number] - dps = delta.New[data.Number]() +type Pipeline[D data.Point[D]] struct { + aggr streams.Aggregator[D] + stale maybe.Ptr[staleness.Staleness[D]] +} + +func pipeline[D data.Point[D]](cfg *Config, tel *telemetry.Telemetry) Pipeline[D] { + var pipe Pipeline[D] + + var dps streams.Map[D] + dps = delta.New[D]() dps = telemetry.ObserveItems(dps, &tel.Metrics) if cfg.MaxStale > 0 { - tel.WithStale(meter, cfg.MaxStale) + tel.WithStale(cfg.MaxStale) stale := maybe.Some(staleness.NewStaleness(cfg.MaxStale, dps)) - proc.stale = stale + pipe.stale = stale dps, _ = stale.Try() } if cfg.MaxStreams > 0 { - tel.WithLimit(meter, int64(cfg.MaxStreams)) + tel.WithLimit(int64(cfg.MaxStreams)) lim := streams.Limit(dps, cfg.MaxStreams) - if stale, ok := proc.stale.Try(); ok { + if stale, ok := pipe.stale.Try(); ok { lim.Evictor = stale } dps = lim @@ -73,13 +87,14 @@ func newProcessor(cfg *Config, log *zap.Logger, meter metric.Meter, next consume dps = telemetry.ObserveNonFatal(dps, &tel.Metrics) - proc.aggr = streams.IntoAggregator(dps) - return &proc + pipe.aggr = streams.IntoAggregator(dps) + return pipe } func (p *Processor) Start(_ context.Context, _ component.Host) error { - stale, ok := p.stale.Try() - if !ok { + sums, sok := p.sums.stale.Try() + expo, eok := p.expo.stale.Try() + if !(sok && eok) { return nil } @@ -91,7 +106,8 @@ func (p *Processor) Start(_ context.Context, _ component.Host) error { return case <-tick.C: p.mtx.Lock() - stale.ExpireOldEntries() + sums.ExpireOldEntries() + expo.ExpireOldEntries() p.mtx.Unlock() } } @@ -119,14 +135,19 @@ func (p *Processor) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) erro case pmetric.MetricTypeSum: sum := m.Sum() if sum.AggregationTemporality() == pmetric.AggregationTemporalityDelta { - err := streams.Aggregate[data.Number](metrics.Sum(m), p.aggr) + err := streams.Aggregate(metrics.Sum(m), p.sums.aggr) errs = errors.Join(errs, err) sum.SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) } case pmetric.MetricTypeHistogram: // TODO case pmetric.MetricTypeExponentialHistogram: - // TODO + expo := m.ExponentialHistogram() + if expo.AggregationTemporality() == pmetric.AggregationTemporalityDelta { + err := streams.Aggregate(metrics.ExpHistogram(m), p.expo.aggr) + errs = errors.Join(errs, err) + expo.SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + } } }) From 8fc7def55e35f2f2eea3ae3ca9b98b0f369ecade Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 22 Apr 2024 22:31:48 +0200 Subject: [PATCH 11/24] *: addlicense, goporto --- processor/deltatocumulativeprocessor/go.mod | 2 ++ .../deltatocumulativeprocessor/internal/data/expo/expo.go | 5 ++++- .../internal/data/expo/expo_test.go | 3 +++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/processor/deltatocumulativeprocessor/go.mod b/processor/deltatocumulativeprocessor/go.mod index eba98afa769f..c9d814ade430 100644 --- a/processor/deltatocumulativeprocessor/go.mod +++ b/processor/deltatocumulativeprocessor/go.mod @@ -63,3 +63,5 @@ replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest => ../../pkg/pdatatest replace github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics => ../../internal/exp/metrics + +replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden => ../../pkg/golden diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go index 7e9ce72c7c9a..488377354135 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go @@ -1,4 +1,7 @@ -package expo +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import "go.opentelemetry.io/collector/pdata/pmetric" diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go index 65e4368ff190..b54939228587 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package expo_test import ( From 7e6576dd37044e3a4ebf74fe36537235a355bd33 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 24 Apr 2024 19:25:15 +0200 Subject: [PATCH 12/24] expotest: reflect getter comparison uses reflect to compare getters and exported fields of structs. this gives a pretty good comparison of OTel types and is much more readable than testifys output. if no diff is found, assert.Equal is also invoked, just to be sure --- processor/deltatocumulativeprocessor/go.mod | 1 - processor/deltatocumulativeprocessor/go.sum | 4 - .../internal/data/data.go | 3 +- .../internal/data/expo/expo_test.go | 7 +- .../internal/data/expo/expotest/equal.go | 98 ++++++++++++++----- .../internal/data/expo/scale_test.go | 10 +- 6 files changed, 86 insertions(+), 37 deletions(-) diff --git a/processor/deltatocumulativeprocessor/go.mod b/processor/deltatocumulativeprocessor/go.mod index c9d814ade430..bf405d5c05a0 100644 --- a/processor/deltatocumulativeprocessor/go.mod +++ b/processor/deltatocumulativeprocessor/go.mod @@ -3,7 +3,6 @@ module github.com/open-telemetry/opentelemetry-collector-contrib/processor/delta go 1.21.0 require ( - github.com/matryer/is v1.4.1 github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.98.0 github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.97.0 github.com/stretchr/testify v1.9.0 diff --git a/processor/deltatocumulativeprocessor/go.sum b/processor/deltatocumulativeprocessor/go.sum index 2b99a6867570..2a01d5d622d2 100644 --- a/processor/deltatocumulativeprocessor/go.sum +++ b/processor/deltatocumulativeprocessor/go.sum @@ -33,8 +33,6 @@ github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/matryer/is v1.4.1 h1:55ehd8zaGABKLXQUe2awZ99BD/PTc2ls+KV/dXphgEQ= -github.com/matryer/is v1.4.1/go.mod h1:8I/i5uYgLzgsgEloJE1U6xx5HkBQpAZvepWuujKwMRU= github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= @@ -44,8 +42,6 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= -github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.98.0 h1:Y4UbYtpt+AsCVj4W8HKxLSlKNrMLhxypmA34GkwjJGE= -github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.98.0/go.mod h1:a1eqRzzePACd9YamtiY0+42P4dn1pB0Ozc0B3P6JGMs= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= diff --git a/processor/deltatocumulativeprocessor/internal/data/data.go b/processor/deltatocumulativeprocessor/internal/data/data.go index 0e5254e02034..eade94eadf92 100644 --- a/processor/deltatocumulativeprocessor/internal/data/data.go +++ b/processor/deltatocumulativeprocessor/internal/data/data.go @@ -4,9 +4,10 @@ package data // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" import ( - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type Point[Self any] interface { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go index b54939228587..8765d0ed965d 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go @@ -7,20 +7,19 @@ import ( "fmt" "testing" - "github.com/matryer/is" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" ) func TestAbsolute(t *testing.T) { - is := is.NewRelaxed(t) + is := expotest.Is(t) bs := expotest.Bins{ø, 1, 2, 3, 4, 5, ø, ø}.Into() abs := expo.Abs(bs) lo, up := abs.Lower(), abs.Upper() - is.Equal(-2, lo) // lower bound - is.Equal(3, up) // upper bound + is.Equalf(-2, lo, "lower-bound") + is.Equalf(3, up, "upper-bound") for i := lo; i < up; i++ { got := abs.Abs(i) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index 3c6bf86b66c2..fedd417a9536 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -4,41 +4,95 @@ package expotest // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" import ( + "reflect" + "strings" "testing" - "github.com/matryer/is" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) type I struct { - *is.I + t *testing.T } func Is(t *testing.T) *I { - return &I{I: is.NewRelaxed(t)} + return &I{t: t} } func (is *I) Equal(want, got any) { - switch got := got.(type) { - case expo.DataPoint: - want := want.(expo.DataPoint) - is.I.Equal(want.Timestamp(), got.Timestamp()) // time - is.I.Equal(want.StartTimestamp(), got.StartTimestamp()) // start-time - is.I.Equal(want.Scale(), got.Scale()) // scale - is.I.Equal(want.Count(), got.Count()) // count - is.I.Equal(want.Sum(), got.Sum()) // sum - is.I.Equal(want.ZeroCount(), got.ZeroCount()) // zero-count - is.I.Equal(want.ZeroThreshold(), got.ZeroThreshold()) // zero-threshold - is.Equal(want.Positive(), got.Positive()) - is.Equal(want.Negative(), got.Negative()) - is.NoErr(pmetrictest.CompareExponentialHistogramDataPoint(want, got)) // pmetrictest - case expo.Buckets: - want := want.(expo.Buckets) - is.I.Equal(want.Offset(), got.Offset()) // offset - is.I.Equal(want.BucketCounts().AsRaw(), got.BucketCounts().AsRaw()) // counts - default: - is.I.Equal(got, want) + is.t.Helper() + equal(is.t, want, got, "") +} + +func (is *I) Equalf(want, got any, name string) { + is.t.Helper() + equal(is.t, want, got, name) +} + +func equal(t *testing.T, want, got any, name string) bool { + t.Helper() + require.IsType(t, want, got) + + vw := reflect.ValueOf(want) + vg := reflect.ValueOf(got) + + if vw.Kind() != reflect.Struct { + ok := reflect.DeepEqual(want, got) + if !ok { + t.Errorf("%s: %+v != %+v", name, want, got) + } + return ok + } + + ok := true + // compare all "getters" of the struct + for i := 0; i < vw.NumMethod(); i++ { + mname := vw.Type().Method(i).Name + name := strings.TrimPrefix(name+"."+mname+"()", ".") + + mw := vw.Method(i) + mg := vg.Method(i) + + // only compare "getters" + if mw.Type().NumIn() != 0 || mw.Type().NumOut() != 1 { + continue + } + // Append(Empty) fails above heuristic, exclude it + if strings.HasPrefix(mname, "Append") { + continue + } + + rw := mw.Call(nil)[0].Interface() + rg := mg.Call(nil)[0].Interface() + + ok = equal(t, rw, rg, name) && ok + } + + // compare all exported fields of the struct + for i := 0; i < vw.NumField(); i++ { + if !vw.Type().Field(i).IsExported() { + continue + } + name := name + "." + vw.Type().Field(i).Name + fw := vw.Field(i).Interface() + fg := vg.Field(i).Interface() + ok = equal(t, fw, fg, name) && ok } + if !ok { + return false + } + + if _, ok := want.(expo.DataPoint); ok { + err := pmetrictest.CompareExponentialHistogramDataPoint(want.(expo.DataPoint), got.(expo.DataPoint)) + if err != nil { + t.Error(err) + } + } + + // fallback to a full deep-equal for rare cases (unexported fields, etc) + return assert.Equal(t, want, got) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go index 83d9b9f4ce9b..6f53e5b0c841 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -9,10 +9,10 @@ import ( "strings" "testing" - "github.com/matryer/is" "go.opentelemetry.io/collector/pdata/pmetric" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" ) func TestDownscale(t *testing.T) { @@ -66,17 +66,17 @@ func TestDownscale(t *testing.T) { buckets[i] = Repr[B]{scale: r.scale, bkt: bkt} } - is := is.NewRelaxed(t) + is := expotest.Is(t) for i := 0; i < len(buckets)-1; i++ { expo.Downscale(buckets[i].bkt, buckets[i].scale, buckets[i+1].scale) - is.Equal(buckets[i+1].bkt.Offset(), buckets[i].bkt.Offset()) // offset must be equal + is.Equalf(buckets[i+1].bkt.Offset(), buckets[i].bkt.Offset(), "offset") want := buckets[i+1].bkt.BucketCounts().AsRaw() got := buckets[i].bkt.BucketCounts().AsRaw() - is.Equal(want, got[:len(want)]) // counts must be equal - is.Equal(make([]uint64, len(got)-len(want)), got[len(want):]) // extra space must be zero + is.Equalf(want, got[:len(want)], "counts") + is.Equalf(make([]uint64, len(got)-len(want)), got[len(want):], "extra-space") } }) } From 50c6f6ab23fabbb0aa20dc7d4d7a0031fa1439e1 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 24 Apr 2024 19:29:26 +0200 Subject: [PATCH 13/24] *: make linter happy I like shadowing, the linter does not. The linter wins today --- .../internal/data/expo/expotest/equal.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index fedd417a9536..75838116801a 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -52,7 +52,7 @@ func equal(t *testing.T, want, got any, name string) bool { // compare all "getters" of the struct for i := 0; i < vw.NumMethod(); i++ { mname := vw.Type().Method(i).Name - name := strings.TrimPrefix(name+"."+mname+"()", ".") + fname := strings.TrimPrefix(name+"."+mname+"()", ".") mw := vw.Method(i) mg := vg.Method(i) @@ -69,7 +69,7 @@ func equal(t *testing.T, want, got any, name string) bool { rw := mw.Call(nil)[0].Interface() rg := mg.Call(nil)[0].Interface() - ok = equal(t, rw, rg, name) && ok + ok = equal(t, rw, rg, fname) && ok } // compare all exported fields of the struct @@ -77,10 +77,10 @@ func equal(t *testing.T, want, got any, name string) bool { if !vw.Type().Field(i).IsExported() { continue } - name := name + "." + vw.Type().Field(i).Name + fname := name + "." + vw.Type().Field(i).Name fw := vw.Field(i).Interface() fg := vg.Field(i).Interface() - ok = equal(t, fw, fg, name) && ok + ok = equal(t, fw, fg, fname) && ok } if !ok { return false From 154a7d70f38e5ec3d00d0164bcb1f97d46badfec Mon Sep 17 00:00:00 2001 From: sh0rez Date: Wed, 24 Apr 2024 19:32:41 +0200 Subject: [PATCH 14/24] *: changelog --- .chloggen/deltatocumulative-exphist.yaml | 29 ++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 .chloggen/deltatocumulative-exphist.yaml diff --git a/.chloggen/deltatocumulative-exphist.yaml b/.chloggen/deltatocumulative-exphist.yaml new file mode 100644 index 000000000000..1bfd68f37258 --- /dev/null +++ b/.chloggen/deltatocumulative-exphist.yaml @@ -0,0 +1,29 @@ +# Use this changelog template to create an entry for release notes. + +# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix' +change_type: "enhancement" + +# The name of the component, or a single word describing the area of concern, (e.g. filelogreceiver) +component: deltatocumulativeprocessor + +# A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`). +note: exponential histograms + +# Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists. +issues: [31340] + +# (Optional) One or more lines of additional information to render under the primary note. +# These lines will be padded with 2 spaces and then inserted directly into the document. +# Use pipe (|) for multiline entries. +subtext: + accumulates exponential histogram datapoints by adding respective bucket counts. + also handles downscaling, changing zero-counts, offset adaptions and optional fields + +# If your change doesn't affect end users or the exported elements of any package, +# you should instead start your pull request title with [chore] or use the "Skip Changelog" label. +# Optional: The change log or logs in which this entry should be included. +# e.g. '[user]' or '[user, api]' +# Include 'user' if the change is relevant to end users. +# Include 'api' if there is a change to a library API. +# Default: '[user]' +change_logs: [user] From 7c14cf4e045c1179ebed314f3c4f1504fa73f8ed Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 29 Apr 2024 16:55:54 +0200 Subject: [PATCH 15/24] deltatocumulative: scope panics to component Uses new `fatal.Context` to recover panics and cancel the context, leading to a controlled failure state --- .../internal/fatal/fatal.go | 85 +++++++++++++++++++ .../internal/fatal/fatal_test.go | 33 +++++++ .../deltatocumulativeprocessor/processor.go | 16 ++-- 3 files changed, 129 insertions(+), 5 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/fatal/fatal.go create mode 100644 processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal.go new file mode 100644 index 000000000000..a32bb6e16434 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/fatal/fatal.go @@ -0,0 +1,85 @@ +// fatal scopes panics to a [context.Context] that is canceled when one occurs. +// +// This allows the use of panic for assertions / fatal faults that cannot be +// recovered from without affecting other parts of the application. +// +// Users are expected to: +// - defer [fatal.Recover] in every goroutine of the same error scope +// - check [fatal.Failed] or [context.Cause] before performing any work and +// abort if a fatal fault has occured +package fatal + +import ( + "context" + "fmt" + "os" + "runtime/debug" + "time" +) + +var now = time.Now + +// scope defines a fault scope that is canceled on panic +type scope struct { + fail func(error) +} + +type ctxkey struct{} + +// Context returns a regular [context.Context] that can also be used to record +// fatal faults using [fatal.Recover]. +// +// All operations of this fault scope are expected to check [context.Cause](ctx) +// or [fatal.Failed](ctx) before performing work. +func Context(parent context.Context) (ctx context.Context, cancel context.CancelCauseFunc) { + ctx, cancel = context.WithCancelCause(parent) + state := &scope{fail: cancel} + ctx = context.WithValue(ctx, ctxkey{}, state) + return ctx, cancel +} + +func from(ctx context.Context) *scope { + if s, ok := ctx.Value(ctxkey{}).(*scope); ok { + return s + } + + return &scope{fail: func(error) { + panic("fatal.Recover must be used with a ctx from fatal.Context") + }} +} + +// Failed reports whether any fatal fault has occured in this fault scope +// If true, no further work must be performed +func Failed(ctx context.Context) bool { + _, ok := context.Cause(ctx).(Error) + return ok +} + +// Recover consumes any later panic. In such a it prints a stacktrace and +// cancels ctx +// +// Must be deffered at the start of each goroutine of the same fault scope: +// +// defer fatal.Recover(ctx) +func Recover(ctx context.Context) { + r := recover() + if r == nil { + return + } + + state := from(ctx) + state.fail(Error{Time: now()}) + + fmt.Println("fatal:", r, "[recovered]") + os.Stdout.Write(debug.Stack()) + fmt.Println("ceasing operations") +} + +// Error signals a fatal fault happened at given Time. +type Error struct { + Time time.Time +} + +func (e Error) Error() string { + return fmt.Sprintf("ceased operations due to a fatal error at %s", e.Time) +} diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go new file mode 100644 index 000000000000..98399b2e348e --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go @@ -0,0 +1,33 @@ +package fatal + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestRecover(t *testing.T) { + ts := time.Date(1234, 1, 2, 3, 4, 0, 0, time.UTC) + now = func() time.Time { + return ts + } + + ctx, cancel := Context(context.Background()) + defer cancel(nil) + + defer func() { + require.Equal(t, true, Failed(ctx)) + require.Equal(t, context.Canceled, ctx.Err()) + require.Equal(t, Error{Time: ts}, context.Cause(ctx)) + }() + + defer Recover(ctx) + require.Equal(t, false, Failed(ctx)) + + func() { + panic("this is bad") + }() + +} diff --git a/processor/deltatocumulativeprocessor/processor.go b/processor/deltatocumulativeprocessor/processor.go index de4680fb66e8..bd520dc04134 100644 --- a/processor/deltatocumulativeprocessor/processor.go +++ b/processor/deltatocumulativeprocessor/processor.go @@ -19,6 +19,7 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics/staleness" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/delta" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/fatal" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/maybe" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/metrics" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/streams" @@ -32,7 +33,7 @@ type Processor struct { log *zap.Logger ctx context.Context - cancel context.CancelFunc + cancel context.CancelCauseFunc sums Pipeline[data.Number] expo Pipeline[data.ExpHistogram] @@ -41,7 +42,7 @@ type Processor struct { } func newProcessor(cfg *Config, log *zap.Logger, meter metric.Meter, next consumer.Metrics) *Processor { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := fatal.Context(context.Background()) tel := telemetry.New(meter) @@ -99,6 +100,8 @@ func (p *Processor) Start(_ context.Context, _ component.Host) error { } go func() { + fatal.Recover(p.ctx) + tick := time.NewTicker(time.Minute) for { select { @@ -116,7 +119,7 @@ func (p *Processor) Start(_ context.Context, _ component.Host) error { } func (p *Processor) Shutdown(_ context.Context) error { - p.cancel() + p.cancel(errors.New("shutdown")) return nil } @@ -125,11 +128,15 @@ func (p *Processor) Capabilities() consumer.Capabilities { } func (p *Processor) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + defer fatal.Recover(p.ctx) + if err := context.Cause(p.ctx); err != nil { + return err + } + p.mtx.Lock() defer p.mtx.Unlock() var errs error - metrics.Each(md, func(m metrics.Metric) { switch m.Type() { case pmetric.MetricTypeSum: @@ -150,7 +157,6 @@ func (p *Processor) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) erro } } }) - if errs != nil { return errs } From d3a9c57c3cb9e1d5b4951a6a1dfe41791c2d475f Mon Sep 17 00:00:00 2001 From: sh0rez Date: Thu, 2 May 2024 15:58:22 +0200 Subject: [PATCH 16/24] *: review feedback --- .../internal/data/add.go | 38 ++++----- .../internal/data/expo/expo.go | 8 +- .../internal/data/expo/expo_test.go | 30 +++++-- .../internal/data/expo/expotest/equal.go | 39 +++++++--- .../internal/data/expo/expotest/equal_test.go | 78 +++++++++++++++++++ .../internal/data/expo/ord.go | 14 ++-- .../internal/data/expo/ord_test.go | 40 ++++++++++ .../internal/data/expo/scale.go | 2 + .../internal/data/expo/zero.go | 6 +- .../internal/fatal/fatal.go | 5 +- .../internal/fatal/fatal_test.go | 3 + 11 files changed, 210 insertions(+), 53 deletions(-) create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go create mode 100644 processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go diff --git a/processor/deltatocumulativeprocessor/internal/data/add.go b/processor/deltatocumulativeprocessor/internal/data/add.go index 75955f92dba1..94a575b1bd9f 100644 --- a/processor/deltatocumulativeprocessor/internal/data/add.go +++ b/processor/deltatocumulativeprocessor/internal/data/add.go @@ -31,9 +31,6 @@ func (dp Histogram) Add(in Histogram) Histogram { func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { type H = ExpHistogram - if dp.Timestamp() >= in.Timestamp() { - panic("out of order") - } if dp.Scale() != in.Scale() { hi, lo := expo.HiLo(dp, in, H.Scale) @@ -44,7 +41,7 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { } if dp.ZeroThreshold() != in.ZeroThreshold() { - hi, lo := expo.HiLo(dp, in, ExpHistogram.ZeroThreshold) + hi, lo := expo.HiLo(dp, in, H.ZeroThreshold) expo.WidenZero(lo.DataPoint, hi.ZeroThreshold()) } @@ -55,26 +52,23 @@ func (dp ExpHistogram) Add(in ExpHistogram) ExpHistogram { dp.SetCount(dp.Count() + in.Count()) dp.SetZeroCount(dp.ZeroCount() + in.ZeroCount()) - optionals := []field{ - {get: H.Sum, set: H.SetSum, has: H.HasSum, del: H.RemoveSum, op: func(a, b float64) float64 { return a + b }}, - {get: H.Min, set: H.SetMin, has: H.HasMin, del: H.RemoveMin, op: math.Min}, - {get: H.Max, set: H.SetMax, has: H.HasMax, del: H.RemoveMax, op: math.Max}, + if dp.HasSum() && in.HasSum() { + dp.SetSum(dp.Sum() + in.Sum()) + } else { + dp.RemoveSum() } - for _, f := range optionals { - if f.has(dp) && f.has(in) { - f.set(dp, f.op(f.get(dp), f.get(in))) - } else { - f.del(dp) - } + + if dp.HasMin() && in.HasMin() { + dp.SetMin(math.Min(dp.Min(), in.Min())) + } else { + dp.RemoveMin() } - return dp -} + if dp.HasMax() && in.HasMax() { + dp.SetMax(math.Max(dp.Max(), in.Max())) + } else { + dp.RemoveMax() + } -type field struct { - get func(ExpHistogram) float64 - set func(ExpHistogram, float64) - has func(ExpHistogram) bool - del func(ExpHistogram) - op func(a, b float64) float64 + return dp } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go index 488377354135..2011e3cd811e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo.go @@ -1,6 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 +// Package expo implements various operations on exponential histograms and their bucket counts package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import "go.opentelemetry.io/collector/pdata/pmetric" @@ -10,8 +11,9 @@ type ( Buckets = pmetric.ExponentialHistogramDataPointBuckets ) -func Abs(buckets buckets) Absolute { - return Absolute{buckets: buckets} +// Abs returns a view into the buckets using an absolute scale +func Abs(bs Buckets) Absolute { + return Absolute{buckets: bs} } type buckets = Buckets @@ -20,6 +22,8 @@ type buckets = Buckets // interoperable with [Scale]. // // It spans from [[Absolute.Lower]:[Absolute.Upper]] +// +// NOTE: The zero-value is unusable, use [Abs] to construct type Absolute struct { buckets } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go index 8765d0ed965d..d7eb0cb2e9b3 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expo_test.go @@ -33,17 +33,31 @@ func ExampleAbsolute() { bs := expotest.Observe0(nums...) abs := expo.Abs(bs) - fmt.Printf("spans from %d:%d\n", abs.Lower(), abs.Upper()) - s := expo.Scale(0) for _, n := range nums { - fmt.Printf("%.1f belongs to bucket %2d: %d\n", n, s.Idx(n), abs.Abs(s.Idx(n))) + fmt.Printf("%.1f belongs to bucket %+d\n", n, s.Idx(n)) + } + + fmt.Printf("\n index:") + for i := 0; i < bs.BucketCounts().Len(); i++ { + fmt.Printf(" %d", i) + } + fmt.Printf("\n abs:") + for i := abs.Lower(); i < abs.Upper(); i++ { + fmt.Printf(" %+d", i) + } + fmt.Printf("\ncounts:") + for i := abs.Lower(); i < abs.Upper(); i++ { + fmt.Printf(" %d", abs.Abs(i)) } // Output: - // spans from -2:3 - // 0.4 belongs to bucket -2: 1 - // 2.3 belongs to bucket 1: 2 - // 2.4 belongs to bucket 1: 2 - // 4.5 belongs to bucket 2: 1 + // 0.4 belongs to bucket -2 + // 2.3 belongs to bucket +1 + // 2.4 belongs to bucket +1 + // 4.5 belongs to bucket +2 + // + // index: 0 1 2 3 4 + // abs: -2 -1 +0 +1 +2 + // counts: 1 0 0 2 1 } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go index 75838116801a..c34e7c1665bc 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal.go @@ -15,25 +15,42 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" ) -type I struct { - t *testing.T +// T is the testing helper. Most notably it provides [T.Equal] +type T struct { + testing.TB } -func Is(t *testing.T) *I { - return &I{t: t} +func Is(t testing.TB) T { + return T{TB: t} } -func (is *I) Equal(want, got any) { - is.t.Helper() - equal(is.t, want, got, "") +// Equal reports whether want and got are deeply equal. +// +// Unlike [reflect.DeepEqual] it first recursively checks exported fields +// and "getters", which are defined as an exported method with: +// - exactly zero input arguments +// - exactly one return value +// - does not start with 'Append' +// +// If this yields differences, those are reported and the test fails. +// If the compared values are [pmetric.ExponentialHistogramDataPoint], then +// [pmetrictest.CompareExponentialHistogramDataPoint] is also called. +// +// If no differences are found, it falls back to [assert.Equal]. +// +// This was done to aid readability when comparing deeply nested [pmetric]/[pcommon] types, +// because in many cases [assert.Equal] output was found to be barely understandable. +func (is T) Equal(want, got any) { + is.Helper() + equal(is.TB, want, got, "") } -func (is *I) Equalf(want, got any, name string) { - is.t.Helper() - equal(is.t, want, got, name) +func (is T) Equalf(want, got any, name string) { + is.Helper() + equal(is.TB, want, got, name) } -func equal(t *testing.T, want, got any, name string) bool { +func equal(t testing.TB, want, got any, name string) bool { t.Helper() require.IsType(t, want, got) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go new file mode 100644 index 000000000000..45dec46159e2 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go @@ -0,0 +1,78 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expotest + +import ( + "fmt" + "path/filepath" + "runtime" + "strconv" + "strings" + "testing" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" +) + +var t testing.TB = fakeT{} + +var expotest = struct { + Is func(t testing.TB) T + Observe func(expo.Scale, ...float64) expo.Buckets +}{ + Is: Is, + Observe: Observe, +} + +func ExampleT_Equal() { + is := expotest.Is(t) + + want := Histogram{ + PosNeg: expotest.Observe(expo.Scale(0), 1, 2, 3, 4), + Scale: 0, + }.Into() + + got := Histogram{ + PosNeg: expotest.Observe(expo.Scale(1), 1, 1, 1, 1), + Scale: 1, + }.Into() + + is.Equal(want, got) + // Output: + // equal_test.go:40: Negative().BucketCounts().AsRaw(): [1 1 2] != [4] + // equal_test.go:40: Negative().BucketCounts().Len(): 3 != 1 + // equal_test.go:40: Positive().BucketCounts().AsRaw(): [1 1 2] != [4] + // equal_test.go:40: Positive().BucketCounts().Len(): 3 != 1 + // equal_test.go:40: Scale(): 0 != 1 +} + +func TestNone(real *testing.T) { + t = real + ExampleT_Equal() + t = fakeT{} +} + +type fakeT struct { + testing.TB +} + +func (t fakeT) Helper() { + return +} + +func (t fakeT) Errorf(format string, args ...any) { + var from string + for i := 0; ; i++ { + pc, file, line, ok := runtime.Caller(i) + if !ok { + break + } + fn := runtime.FuncForPC(pc) + if strings.HasSuffix(fn.Name(), ".ExampleT_Equal") { + from = filepath.Base(file) + ":" + strconv.Itoa(line) + break + } + } + + fmt.Printf("%s: %s\n", from, fmt.Sprintf(format, args...)) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/ord.go b/processor/deltatocumulativeprocessor/internal/data/expo/ord.go index af5527b44244..34d177be1795 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/ord.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/ord.go @@ -3,16 +3,14 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" -type ord interface { - int | int32 | float64 -} +import "cmp" // HiLo returns the greater of a and b by comparing the result of applying fn to -// each -func HiLo[T any, N ord](a, b T, fn func(T) N) (hi, lo T) { +// each. If equal, returns operands as passed +func HiLo[T any, N cmp.Ordered](a, b T, fn func(T) N) (hi, lo T) { an, bn := fn(a), fn(b) - if an > bn { - return a, b + if cmp.Less(an, bn) { + return b, a } - return b, a + return a, b } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go new file mode 100644 index 000000000000..155ac3fa94a2 --- /dev/null +++ b/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go @@ -0,0 +1,40 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package expo_test + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" +) + +func TestHiLo(t *testing.T) { + type T struct { + int int + str string + } + + a := T{int: 0, str: "foo"} + b := T{int: 1, str: "bar"} + + { + hi, lo := expo.HiLo(a, b, func(v T) int { return v.int }) + assert.Equal(t, a, lo) + assert.Equal(t, b, hi) + } + + { + hi, lo := expo.HiLo(a, b, func(v T) string { return v.str }) + assert.Equal(t, b, lo) + assert.Equal(t, a, hi) + } + + { + hi, lo := expo.HiLo(a, b, func(v T) int { return 0 }) + assert.Equal(t, a, hi) + assert.Equal(t, b, lo) + } +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index 69f7747ad1c0..37466b688940 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -56,6 +56,8 @@ func Downscale(bs Buckets, from, to Scale) { // // before: 1 1 1 1 1 1 1 1 1 1 1 1 // after: 2 2 2 2 2 2 0 0 0 0 0 0 +// +// The result is equal to the same observation with the scale lowered by 1 func Collapse(bs Buckets) { counts := bs.BucketCounts() size := counts.Len() / 2 diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go index 37e27420e18e..e956d3a69869 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -10,7 +10,7 @@ import ( // WidenZero widens the zero-bucket to span at least [-width,width], possibly wider // if min falls in the middle of a bucket. // -// Both buckets MUST be of same scale. +// Both buckets counts MUST be of same scale. func WidenZero(dp DataPoint, width float64) { switch { case width == dp.ZeroThreshold(): @@ -39,6 +39,10 @@ func WidenZero(dp DataPoint, width float64) { } // Slice drops data outside the range from <= i < to from the bucket counts. It behaves the same as Go's [a:b] +// +// Limitations: +// - due to a limitation of the pcommon package, slicing cannot happen in-place and allocates +// - in consequence, data outside the range is garbage collected func (a Absolute) Slice(from, to int) { lo, up := a.Lower(), a.Upper() switch { diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal.go index a32bb6e16434..5bf26b5cff4a 100644 --- a/processor/deltatocumulativeprocessor/internal/fatal/fatal.go +++ b/processor/deltatocumulativeprocessor/internal/fatal/fatal.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + // fatal scopes panics to a [context.Context] that is canceled when one occurs. // // This allows the use of panic for assertions / fatal faults that cannot be @@ -7,7 +10,7 @@ // - defer [fatal.Recover] in every goroutine of the same error scope // - check [fatal.Failed] or [context.Cause] before performing any work and // abort if a fatal fault has occured -package fatal +package fatal // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/fatal" import ( "context" diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go index 98399b2e348e..3866b166dbe1 100644 --- a/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go +++ b/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package fatal import ( From b11184b1561ae648d400dcb8b9284b99fc3c3714 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Thu, 2 May 2024 16:00:17 +0200 Subject: [PATCH 17/24] changelog: reword --- .chloggen/deltatocumulative-exphist.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.chloggen/deltatocumulative-exphist.yaml b/.chloggen/deltatocumulative-exphist.yaml index 1bfd68f37258..7dfa30bf54e4 100644 --- a/.chloggen/deltatocumulative-exphist.yaml +++ b/.chloggen/deltatocumulative-exphist.yaml @@ -7,7 +7,7 @@ change_type: "enhancement" component: deltatocumulativeprocessor # A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`). -note: exponential histograms +note: exponential histogram accumulation # Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists. issues: [31340] From 55f3bbc39c9b29151f0df7fe964ad9772ebc3290 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 6 May 2024 15:24:59 +0200 Subject: [PATCH 18/24] *: remove fatal recovery agreed during review this was not overly useful and added complexity. will unit-test the panics instead --- .../internal/fatal/fatal.go | 88 ------------------- .../internal/fatal/fatal_test.go | 36 -------- .../deltatocumulativeprocessor/processor.go | 10 +-- 3 files changed, 3 insertions(+), 131 deletions(-) delete mode 100644 processor/deltatocumulativeprocessor/internal/fatal/fatal.go delete mode 100644 processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal.go deleted file mode 100644 index 5bf26b5cff4a..000000000000 --- a/processor/deltatocumulativeprocessor/internal/fatal/fatal.go +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -// fatal scopes panics to a [context.Context] that is canceled when one occurs. -// -// This allows the use of panic for assertions / fatal faults that cannot be -// recovered from without affecting other parts of the application. -// -// Users are expected to: -// - defer [fatal.Recover] in every goroutine of the same error scope -// - check [fatal.Failed] or [context.Cause] before performing any work and -// abort if a fatal fault has occured -package fatal // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/fatal" - -import ( - "context" - "fmt" - "os" - "runtime/debug" - "time" -) - -var now = time.Now - -// scope defines a fault scope that is canceled on panic -type scope struct { - fail func(error) -} - -type ctxkey struct{} - -// Context returns a regular [context.Context] that can also be used to record -// fatal faults using [fatal.Recover]. -// -// All operations of this fault scope are expected to check [context.Cause](ctx) -// or [fatal.Failed](ctx) before performing work. -func Context(parent context.Context) (ctx context.Context, cancel context.CancelCauseFunc) { - ctx, cancel = context.WithCancelCause(parent) - state := &scope{fail: cancel} - ctx = context.WithValue(ctx, ctxkey{}, state) - return ctx, cancel -} - -func from(ctx context.Context) *scope { - if s, ok := ctx.Value(ctxkey{}).(*scope); ok { - return s - } - - return &scope{fail: func(error) { - panic("fatal.Recover must be used with a ctx from fatal.Context") - }} -} - -// Failed reports whether any fatal fault has occured in this fault scope -// If true, no further work must be performed -func Failed(ctx context.Context) bool { - _, ok := context.Cause(ctx).(Error) - return ok -} - -// Recover consumes any later panic. In such a it prints a stacktrace and -// cancels ctx -// -// Must be deffered at the start of each goroutine of the same fault scope: -// -// defer fatal.Recover(ctx) -func Recover(ctx context.Context) { - r := recover() - if r == nil { - return - } - - state := from(ctx) - state.fail(Error{Time: now()}) - - fmt.Println("fatal:", r, "[recovered]") - os.Stdout.Write(debug.Stack()) - fmt.Println("ceasing operations") -} - -// Error signals a fatal fault happened at given Time. -type Error struct { - Time time.Time -} - -func (e Error) Error() string { - return fmt.Sprintf("ceased operations due to a fatal error at %s", e.Time) -} diff --git a/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go b/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go deleted file mode 100644 index 3866b166dbe1..000000000000 --- a/processor/deltatocumulativeprocessor/internal/fatal/fatal_test.go +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package fatal - -import ( - "context" - "testing" - "time" - - "github.com/stretchr/testify/require" -) - -func TestRecover(t *testing.T) { - ts := time.Date(1234, 1, 2, 3, 4, 0, 0, time.UTC) - now = func() time.Time { - return ts - } - - ctx, cancel := Context(context.Background()) - defer cancel(nil) - - defer func() { - require.Equal(t, true, Failed(ctx)) - require.Equal(t, context.Canceled, ctx.Err()) - require.Equal(t, Error{Time: ts}, context.Cause(ctx)) - }() - - defer Recover(ctx) - require.Equal(t, false, Failed(ctx)) - - func() { - panic("this is bad") - }() - -} diff --git a/processor/deltatocumulativeprocessor/processor.go b/processor/deltatocumulativeprocessor/processor.go index bd520dc04134..01e1cef4f916 100644 --- a/processor/deltatocumulativeprocessor/processor.go +++ b/processor/deltatocumulativeprocessor/processor.go @@ -19,7 +19,6 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics/staleness" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/delta" - "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/fatal" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/maybe" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/metrics" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/streams" @@ -33,7 +32,7 @@ type Processor struct { log *zap.Logger ctx context.Context - cancel context.CancelCauseFunc + cancel context.CancelFunc sums Pipeline[data.Number] expo Pipeline[data.ExpHistogram] @@ -42,7 +41,7 @@ type Processor struct { } func newProcessor(cfg *Config, log *zap.Logger, meter metric.Meter, next consumer.Metrics) *Processor { - ctx, cancel := fatal.Context(context.Background()) + ctx, cancel := context.WithCancel(context.Background()) tel := telemetry.New(meter) @@ -100,8 +99,6 @@ func (p *Processor) Start(_ context.Context, _ component.Host) error { } go func() { - fatal.Recover(p.ctx) - tick := time.NewTicker(time.Minute) for { select { @@ -119,7 +116,7 @@ func (p *Processor) Start(_ context.Context, _ component.Host) error { } func (p *Processor) Shutdown(_ context.Context) error { - p.cancel(errors.New("shutdown")) + p.cancel() return nil } @@ -128,7 +125,6 @@ func (p *Processor) Capabilities() consumer.Capabilities { } func (p *Processor) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { - defer fatal.Recover(p.ctx) if err := context.Cause(p.ctx); err != nil { return err } From 3c295cff4a1c96bbd66d6e763e2004ef6e0881fa Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 6 May 2024 15:25:46 +0200 Subject: [PATCH 19/24] *: unit test panics --- .../internal/data/expo/scale.go | 6 +++++- .../internal/data/expo/scale_test.go | 7 +++++++ .../internal/data/expo/zero_test.go | 18 ++++++++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index 37466b688940..192ffe3d5064 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -4,6 +4,7 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( + "fmt" "math" ) @@ -44,7 +45,10 @@ func Downscale(bs Buckets, from, to Scale) { case from == to: return case from < to: - panic("cannot upscale without introducing error") + // because even distribution within the buckets cannot be assumed, it is + // not possible to correctly upscale (split) buckets. + // any attempt to do so would yield erronous data. + panic(fmt.Sprintf("cannot upscale without introducing error (%d -> %d)", from, to)) } for at := from; at > to; at-- { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go index 6f53e5b0c841..9631f8838331 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -13,6 +13,7 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" + "github.com/stretchr/testify/assert" ) func TestDownscale(t *testing.T) { @@ -80,4 +81,10 @@ func TestDownscale(t *testing.T) { } }) } + + t.Run("panics", func(t *testing.T) { + assert.PanicsWithValue(t, "cannot upscale without introducing error (8 -> 12)", func() { + expo.Downscale(bins{}.Into(), 8, 12) + }) + }) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go index 4e0e3e56d029..1d98a1c68fc6 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -9,6 +9,7 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" + "github.com/stretchr/testify/assert" ) type hist = expotest.Histogram @@ -54,6 +55,13 @@ func TestWidenZero(t *testing.T) { is.Equal(want, hist) }) } + + t.Run("panics", func(t *testing.T) { + assert.PanicsWithValue(t, "min must be larger than current threshold (1.500000)", func() { + hist := hist{Zt: 1.5}.Into() + expo.WidenZero(hist, 0.5) + }) + }) } func TestSlice(t *testing.T) { @@ -99,4 +107,14 @@ func TestSlice(t *testing.T) { is.Equal(want, bins) }) } + + t.Run("panics", func(t *testing.T) { + data := expo.Abs(bins{1, 2, 3, 4, 5, 6, 7, 8}.Into()) + assert.PanicsWithValue(t, "bad bounds: must be from<=to (got 8<=4)", func() { + data.Slice(8, 4) + }) + assert.PanicsWithValue(t, "-6:12 is out of bounds for -3:5", func() { + data.Slice(-6, 12) + }) + }) } From 47f418cf10b40f5390928d9a65d1e77453cfd604 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 6 May 2024 15:30:22 +0200 Subject: [PATCH 20/24] *: make linter happy --- .../internal/data/expo/expotest/equal_test.go | 5 ++--- .../internal/data/expo/ord_test.go | 2 +- .../internal/data/expo/scale_test.go | 2 +- .../internal/data/expo/zero_test.go | 3 ++- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go index 45dec46159e2..885794b79498 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go @@ -46,6 +46,7 @@ func ExampleT_Equal() { // equal_test.go:40: Scale(): 0 != 1 } +//nolint:predeclared func TestNone(real *testing.T) { t = real ExampleT_Equal() @@ -56,9 +57,7 @@ type fakeT struct { testing.TB } -func (t fakeT) Helper() { - return -} +func (t fakeT) Helper() {} func (t fakeT) Errorf(format string, args ...any) { var from string diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go index 155ac3fa94a2..dedc60b50f27 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/ord_test.go @@ -33,7 +33,7 @@ func TestHiLo(t *testing.T) { } { - hi, lo := expo.HiLo(a, b, func(v T) int { return 0 }) + hi, lo := expo.HiLo(a, b, func(T) int { return 0 }) assert.Equal(t, a, hi) assert.Equal(t, b, lo) } diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go index 9631f8838331..ceb76eb1d44d 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale_test.go @@ -9,11 +9,11 @@ import ( "strings" "testing" + "github.com/stretchr/testify/assert" "go.opentelemetry.io/collector/pdata/pmetric" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" - "github.com/stretchr/testify/assert" ) func TestDownscale(t *testing.T) { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go index 1d98a1c68fc6..bd951b93ca29 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -7,9 +7,10 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/assert" + "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo/expotest" - "github.com/stretchr/testify/assert" ) type hist = expotest.Histogram From 54661a2cb4e5b0947f524fb5f6c1ed769efc3500 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 6 May 2024 16:04:35 +0200 Subject: [PATCH 21/24] expotest: remove unneccesary test case --- .../internal/data/expo/expotest/equal_test.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go index 885794b79498..7fb7c42b586e 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/expotest/equal_test.go @@ -38,6 +38,7 @@ func ExampleT_Equal() { }.Into() is.Equal(want, got) + // Output: // equal_test.go:40: Negative().BucketCounts().AsRaw(): [1 1 2] != [4] // equal_test.go:40: Negative().BucketCounts().Len(): 3 != 1 @@ -46,12 +47,7 @@ func ExampleT_Equal() { // equal_test.go:40: Scale(): 0 != 1 } -//nolint:predeclared -func TestNone(real *testing.T) { - t = real - ExampleT_Equal() - t = fakeT{} -} +func TestNone(*testing.T) {} type fakeT struct { testing.TB From 6be320aa16676ae508142e5f7e959dd8a1da49c1 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Tue, 7 May 2024 14:08:01 +0200 Subject: [PATCH 22/24] expo.WidenZero: properly constraint slice range the zero range may be entirely outside of the populated buckets, in which case we cannot use zerobucket.Upper()+1 as the bucket counts lower bound, because it is outside of the slice range. Properly clamps the slice range to the actual range now --- .../internal/data/expo/zero.go | 18 +++++++++++++----- .../internal/data/expo/zero_test.go | 4 ++++ .../internal/data/expo_test.go | 11 ++++++++--- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go index e956d3a69869..2d5401b39f5c 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero.go @@ -4,6 +4,7 @@ package expo // import "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor/internal/data/expo" import ( + "cmp" "fmt" ) @@ -20,21 +21,23 @@ func WidenZero(dp DataPoint, width float64) { } scale := Scale(dp.Scale()) - lo := scale.Idx(width) + zero := scale.Idx(width) // the largest bucket index inside the zero width widen := func(bs Buckets) { abs := Abs(bs) - for i := abs.Lower(); i <= lo; i++ { + for i := abs.Lower(); i <= zero; i++ { dp.SetZeroCount(dp.ZeroCount() + abs.Abs(i)) } - up := abs.Upper() - abs.Slice(min(lo+1, up), up) + + // right next to the new zero bucket, constrained to slice range + lo := clamp(zero+1, abs.Lower(), abs.Upper()) + abs.Slice(lo, abs.Upper()) } widen(dp.Positive()) widen(dp.Negative()) - _, max := scale.Bounds(lo) + _, max := scale.Bounds(zero) dp.SetZeroThreshold(max) } @@ -58,3 +61,8 @@ func (a Absolute) Slice(from, to int) { a.BucketCounts().FromRaw(a.BucketCounts().AsRaw()[first:last]) a.SetOffset(int32(from)) } + +// clamp constraints v to the range up..=lo +func clamp[N cmp.Ordered](v, lo, up N) N { + return max(lo, min(v, up)) +} diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go index bd951b93ca29..92e9d88a38d1 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/zero_test.go @@ -38,6 +38,10 @@ func TestWidenZero(t *testing.T) { hist: hist{PosNeg: bins{ø, ø, 1, 2, 3, 4, 5, ø}.Into(), Zt: 0, Zc: 2}, min: 3, want: hist{PosNeg: bins{ø, ø, ø, ø, ø, 4, 5, ø}.Into(), Zt: 4, Zc: 2 + 2*(1+2+3)}, + }, { + // zt=2 is higher, but no change expected as no buckets in this range are populated + hist: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, 1, 1}.Into(), Zt: 1.0, Zc: 2}, + want: hist{PosNeg: bins{ø, ø, ø, ø, ø, ø, 1, 1}.Into(), Zt: 2.0, Zc: 2}, }} for _, cs := range cases { diff --git a/processor/deltatocumulativeprocessor/internal/data/expo_test.go b/processor/deltatocumulativeprocessor/internal/data/expo_test.go index ad7b4ed359ee..b910b409cb55 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo_test.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo_test.go @@ -63,9 +63,14 @@ func TestAdd(t *testing.T) { want: expdp{PosNeg: bins{0, 2, 2}.Into(), Zt: 0, Zc: 5, Count: 8}, }, { name: "zero/diff", - dp: expdp{Pos: bins{ø, ø, 0, 1, 1, 1}.Into(), Zt: 0.0, Zc: 2}, - in: expdp{Pos: bins{ø, ø, ø, ø, 1, 1}.Into(), Zt: 2.0, Zc: 2}, - want: expdp{Pos: bins{ø, ø, ø, ø, 2, 2}.Into(), Zt: 2.0, Zc: 2 + 2 + 1}, + dp: expdp{PosNeg: bins{ø, ø, 0, 1, 1, 1}.Into(), Zt: 0.0, Zc: 2}, + in: expdp{PosNeg: bins{ø, ø, ø, ø, 1, 1}.Into(), Zt: 2.0, Zc: 2}, + want: expdp{PosNeg: bins{ø, ø, ø, ø, 2, 2}.Into(), Zt: 2.0, Zc: 4 + 2*1}, + }, { + name: "zero/subzero", + dp: expdp{PosNeg: bins{ø, 1, 1, 1, 1, 1}.Into(), Zt: 0.2, Zc: 2}, + in: expdp{PosNeg: bins{ø, ø, 1, 1, 1, 1}.Into(), Zt: 0.3, Zc: 2}, + want: expdp{PosNeg: bins{ø, ø, 2, 2, 2, 2}.Into(), Zt: 0.5, Zc: 4 + 2*1}, }, { name: "negative-offset", dp: expdp{PosNeg: rawbs([]uint64{ /* */ 1, 2}, -2)}, From 97cdf13569d587801c03ef8154aadcb9ebc18a9c Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 13 May 2024 23:02:48 +0200 Subject: [PATCH 23/24] expo: Merge only if both are non-zero --- .../internal/data/expo/merge.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go index 04c7d3a90b4e..150e29a65819 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/merge.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/merge.go @@ -10,6 +10,14 @@ import ( // Merge combines the counts of buckets a and b into a. // Both buckets MUST be of same scale func Merge(arel, brel Buckets) { + if brel.BucketCounts().Len() == 0 { + return + } + if arel.BucketCounts().Len() == 0 { + brel.CopyTo(arel) + return + } + a, b := Abs(arel), Abs(brel) lo := min(a.Lower(), b.Lower()) From c80582b07cc773e66ba2cdff3b34d69c414860b5 Mon Sep 17 00:00:00 2001 From: sh0rez Date: Mon, 13 May 2024 23:21:38 +0200 Subject: [PATCH 24/24] expo: more comments for Collapse --- .../internal/data/expo/scale.go | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go index 192ffe3d5064..ac075158dc3c 100644 --- a/processor/deltatocumulativeprocessor/internal/data/expo/scale.go +++ b/processor/deltatocumulativeprocessor/internal/data/expo/scale.go @@ -61,7 +61,14 @@ func Downscale(bs Buckets, from, to Scale) { // before: 1 1 1 1 1 1 1 1 1 1 1 1 // after: 2 2 2 2 2 2 0 0 0 0 0 0 // -// The result is equal to the same observation with the scale lowered by 1 +// Due to the "perfect subsetting" property of exponential histograms, this +// gives the same observation as before, but recorded at scale-1. See +// https://opentelemetry.io/docs/specs/otel/metrics/data-model/#exponential-scale. +// +// Because every bucket now spans twice as much range, half of the allocated +// counts slice is technically no longer required. It is zeroed but left in +// place to avoid future allocations, because observations may happen in that +// area at a later time. func Collapse(bs Buckets) { counts := bs.BucketCounts() size := counts.Len() / 2 @@ -69,6 +76,8 @@ func Collapse(bs Buckets) { size++ } + // merging needs to happen in pairs aligned to i=0. if offset is non-even, + // we need to shift the whole merging by one to make above condition true. shift := 0 if bs.Offset()%2 != 0 { bs.SetOffset(bs.Offset() - 1) @@ -77,19 +86,29 @@ func Collapse(bs Buckets) { bs.SetOffset(bs.Offset() / 2) for i := 0; i < size; i++ { + // size is ~half of len. we add two buckets per iteration. + // k jumps in steps of 2, shifted if offset makes this necessary. k := i*2 + shift + // special case: we just started and had to shift. the left half of the + // new bucket is not actually stored, so only use counts[0]. if i == 0 && k == -1 { counts.SetAt(i, counts.At(k+1)) continue } + // new[k] = old[k]+old[k+1] counts.SetAt(i, counts.At(k)) if k+1 < counts.Len() { counts.SetAt(i, counts.At(k)+counts.At(k+1)) } } + // zero the excess area. its not needed to represent the observation + // anymore, but kept for two reasons: + // 1. future observations may need it, no need to re-alloc then if kept + // 2. [pcommon.Uint64Slice] can not, in fact, be sliced, so getting rid + // of it would alloc ¯\_(ツ)_/¯ for i := size; i < counts.Len(); i++ { counts.SetAt(i, 0) }