Skip to content

Commit

Permalink
[pkg/ottl] Use generics to avoid context cast in getters and funcs (o…
Browse files Browse the repository at this point in the history
…pen-telemetry#14482)

[processor/transform] Added Int function to the transform processor (open-telemetry#14338)

[processor/transform] Added the Int function to the transform processor (open-telemetry#11810)

Co-authored-by: Vitalii Solodilov <[email protected]>
  • Loading branch information
bogdandrutu and mcdoker18 committed Sep 29, 2022
1 parent c5cd3a2 commit e4695ee
Show file tree
Hide file tree
Showing 71 changed files with 1,362 additions and 1,457 deletions.
40 changes: 20 additions & 20 deletions pkg/ottl/boolean_value.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,20 +19,20 @@ import (
)

// boolExpressionEvaluator is a function that returns the result.
type boolExpressionEvaluator = func(ctx TransformContext) bool
type boolExpressionEvaluator[K any] func(ctx K) bool

var alwaysTrue = func(ctx TransformContext) bool {
func alwaysTrue[K any](K) bool {
return true
}

var alwaysFalse = func(ctx TransformContext) bool {
func alwaysFalse[K any](K) bool {
return false
}

// builds a function that returns a short-circuited result of ANDing
// boolExpressionEvaluator funcs
func andFuncs(funcs []boolExpressionEvaluator) boolExpressionEvaluator {
return func(ctx TransformContext) bool {
func andFuncs[K any](funcs []boolExpressionEvaluator[K]) boolExpressionEvaluator[K] {
return func(ctx K) bool {
for _, f := range funcs {
if !f(ctx) {
return false
Expand All @@ -44,8 +44,8 @@ func andFuncs(funcs []boolExpressionEvaluator) boolExpressionEvaluator {

// builds a function that returns a short-circuited result of ORing
// boolExpressionEvaluator funcs
func orFuncs(funcs []boolExpressionEvaluator) boolExpressionEvaluator {
return func(ctx TransformContext) bool {
func orFuncs[K any](funcs []boolExpressionEvaluator[K]) boolExpressionEvaluator[K] {
return func(ctx K) bool {
for _, f := range funcs {
if f(ctx) {
return true
Expand All @@ -55,9 +55,9 @@ func orFuncs(funcs []boolExpressionEvaluator) boolExpressionEvaluator {
}
}

func (p *Parser) newComparisonEvaluator(comparison *Comparison) (boolExpressionEvaluator, error) {
func (p *Parser[K]) newComparisonEvaluator(comparison *Comparison) (boolExpressionEvaluator[K], error) {
if comparison == nil {
return alwaysTrue, nil
return alwaysTrue[K], nil
}
left, err := p.newGetter(comparison.Left)
if err != nil {
Expand All @@ -69,23 +69,23 @@ func (p *Parser) newComparisonEvaluator(comparison *Comparison) (boolExpressionE
}

// The parser ensures that we'll never get an invalid comparison.Op, so we don't have to check that case.
return func(ctx TransformContext) bool {
return func(ctx K) bool {
a := left.Get(ctx)
b := right.Get(ctx)
return p.compare(a, b, comparison.Op)
}, nil

}

func (p *Parser) newBooleanExpressionEvaluator(expr *BooleanExpression) (boolExpressionEvaluator, error) {
func (p *Parser[K]) newBooleanExpressionEvaluator(expr *BooleanExpression) (boolExpressionEvaluator[K], error) {
if expr == nil {
return alwaysTrue, nil
return alwaysTrue[K], nil
}
f, err := p.newBooleanTermEvaluator(expr.Left)
if err != nil {
return nil, err
}
funcs := []boolExpressionEvaluator{f}
funcs := []boolExpressionEvaluator[K]{f}
for _, rhs := range expr.Right {
f, err := p.newBooleanTermEvaluator(rhs.Term)
if err != nil {
Expand All @@ -97,15 +97,15 @@ func (p *Parser) newBooleanExpressionEvaluator(expr *BooleanExpression) (boolExp
return orFuncs(funcs), nil
}

func (p *Parser) newBooleanTermEvaluator(term *Term) (boolExpressionEvaluator, error) {
func (p *Parser[K]) newBooleanTermEvaluator(term *Term) (boolExpressionEvaluator[K], error) {
if term == nil {
return alwaysTrue, nil
return alwaysTrue[K], nil
}
f, err := p.newBooleanValueEvaluator(term.Left)
if err != nil {
return nil, err
}
funcs := []boolExpressionEvaluator{f}
funcs := []boolExpressionEvaluator[K]{f}
for _, rhs := range term.Right {
f, err := p.newBooleanValueEvaluator(rhs.Value)
if err != nil {
Expand All @@ -117,9 +117,9 @@ func (p *Parser) newBooleanTermEvaluator(term *Term) (boolExpressionEvaluator, e
return andFuncs(funcs), nil
}

func (p *Parser) newBooleanValueEvaluator(value *BooleanValue) (boolExpressionEvaluator, error) {
func (p *Parser[K]) newBooleanValueEvaluator(value *BooleanValue) (boolExpressionEvaluator[K], error) {
if value == nil {
return alwaysTrue, nil
return alwaysTrue[K], nil
}
switch {
case value.Comparison != nil:
Expand All @@ -130,9 +130,9 @@ func (p *Parser) newBooleanValueEvaluator(value *BooleanValue) (boolExpressionEv
return comparison, nil
case value.ConstExpr != nil:
if *value.ConstExpr {
return alwaysTrue, nil
return alwaysTrue[K], nil
}
return alwaysFalse, nil
return alwaysFalse[K], nil
case value.SubExpr != nil:
return p.newBooleanExpressionEvaluator(value.SubExpr)
}
Expand Down
56 changes: 26 additions & 30 deletions pkg/ottl/boolean_value_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,45 +85,43 @@ func Test_newComparisonEvaluator(t *testing.T) {
componenttest.NewNopTelemetrySettings(),
)

tests := []struct {
var tests = []struct {
name string
l any
r any
op string
item interface{}
item string
want bool
}{
{"literals match", "hello", "hello", "==", nil, true},
{"literals don't match", "hello", "goodbye", "!=", nil, true},
{"path expression matches", "NAME", "bear", "==", "bear", true},
{"path expression not matches", "NAME", "cat", "!=", "bear", true},
{"compare Enum to int", "TEST_ENUM", int(0), "==", nil, true},
{"compare int to Enum", int(2), "TEST_ENUM_TWO", "==", nil, true},
{"2 > Enum 0", int(2), "TEST_ENUM", ">", nil, true},
{"not 2 < Enum 0", int(2), "TEST_ENUM", "<", nil, false},
{"not 6 == 3.14", 6, 3.14, "==", nil, false},
{"6 != 3.14", 6, 3.14, "!=", nil, true},
{"6 > 3.14", 6, 3.14, ">", nil, true},
{"6 >= 3.14", 6, 3.14, ">=", nil, true},
{"not 6 < 3.14", 6, 3.14, "<", nil, false},
{"not 6 <= 3.14", 6, 3.14, "<=", nil, false},
{"'foo' > 'bar'", "foo", "bar", ">", nil, true},
{"'foo' > bear", "foo", "NAME", ">", "bear", true},
{"true > false", true, false, ">", nil, true},
{"not true > 0", true, 0, ">", nil, false},
{"not 'true' == true", "true", true, "==", nil, false},
{"[]byte('a') < []byte('b')", []byte("a"), []byte("b"), "<", nil, true},
{"nil == nil", nil, nil, "==", nil, true},
{"nil == []byte(nil)", nil, []byte(nil), "==", nil, true},
{name: "literals match", l: "hello", r: "hello", op: "==", want: true},
{name: "literals don't match", l: "hello", r: "goodbye", op: "!=", want: true},
{name: "path expression matches", l: "NAME", r: "bear", op: "==", item: "bear", want: true},
{name: "path expression not matches", l: "NAME", r: "cat", op: "!=", item: "bear", want: true},
{name: "compare Enum to int", l: "TEST_ENUM", r: 0, op: "==", want: true},
{name: "compare int to Enum", l: 2, r: "TEST_ENUM_TWO", op: "==", want: true},
{name: "2 > Enum 0", l: 2, r: "TEST_ENUM", op: ">", want: true},
{name: "not 2 < Enum 0", l: 2, r: "TEST_ENUM", op: "<"},
{name: "not 6 == 3.14", l: 6, r: 3.14, op: "=="},
{name: "6 != 3.14", l: 6, r: 3.14, op: "!=", want: true},
{name: "6 > 3.14", l: 6, r: 3.14, op: ">", want: true},
{name: "6 >= 3.14", l: 6, r: 3.14, op: ">=", want: true},
{name: "not 6 < 3.14", l: 6, r: 3.14, op: "<"},
{name: "not 6 <= 3.14", l: 6, r: 3.14, op: "<="},
{name: "'foo' > 'bar'", l: "foo", r: "bar", op: ">", want: true},
{name: "'foo' > bear", l: "foo", r: "NAME", op: ">", item: "bear", want: true},
{name: "true > false", l: true, r: false, op: ">", want: true},
{name: "not true > 0", l: true, r: 0, op: ">"},
{name: "not 'true' == true", l: "true", r: true, op: "=="},
{name: "[]byte('a') < []byte('b')", l: []byte("a"), r: []byte("b"), op: "<", want: true},
{name: "nil == nil", op: "==", want: true},
{name: "nil == []byte(nil)", r: []byte(nil), op: "==", want: true},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
comp := comparison(tt.l, tt.r, tt.op)
evaluate, err := p.newComparisonEvaluator(comp)
assert.NoError(t, err)
assert.Equal(t, tt.want, evaluate(ottltest.TestTransformContext{
Item: tt.item,
}))
assert.Equal(t, tt.want, evaluate(tt.item))
})
}
}
Expand Down Expand Up @@ -353,9 +351,7 @@ func Test_newBooleanExpressionEvaluator(t *testing.T) {
t.Run(tt.name, func(t *testing.T) {
evaluate, err := p.newBooleanExpressionEvaluator(tt.expr)
assert.NoError(t, err)
assert.Equal(t, tt.want, evaluate(ottltest.TestTransformContext{
Item: nil,
}))
assert.Equal(t, tt.want, evaluate(nil))
})
}
}
14 changes: 7 additions & 7 deletions pkg/ottl/compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
// invalidComparison returns false for everything except NE (where it returns true to indicate that the
// objects were definitely not equivalent).
// It also gives us an opportunity to log something.
func (p *Parser) invalidComparison(msg string, op CompareOp) bool {
func (p *Parser[K]) invalidComparison(msg string, op CompareOp) bool {
p.telemetrySettings.Logger.Debug(msg, zap.Any("op", op))
return op == NE
}
Expand Down Expand Up @@ -92,7 +92,7 @@ func compareBytes(a []byte, b []byte, op CompareOp) bool {
}
}

func (p *Parser) compareBool(a bool, b any, op CompareOp) bool {
func (p *Parser[K]) compareBool(a bool, b any, op CompareOp) bool {
switch v := b.(type) {
case bool:
return compareBools(a, v, op)
Expand All @@ -101,7 +101,7 @@ func (p *Parser) compareBool(a bool, b any, op CompareOp) bool {
}
}

func (p *Parser) compareString(a string, b any, op CompareOp) bool {
func (p *Parser[K]) compareString(a string, b any, op CompareOp) bool {
switch v := b.(type) {
case string:
return comparePrimitives(a, v, op)
Expand All @@ -110,7 +110,7 @@ func (p *Parser) compareString(a string, b any, op CompareOp) bool {
}
}

func (p *Parser) compareByte(a []byte, b any, op CompareOp) bool {
func (p *Parser[K]) compareByte(a []byte, b any, op CompareOp) bool {
switch v := b.(type) {
case nil:
return op == NE
Expand All @@ -124,7 +124,7 @@ func (p *Parser) compareByte(a []byte, b any, op CompareOp) bool {
}
}

func (p *Parser) compareInt64(a int64, b any, op CompareOp) bool {
func (p *Parser[K]) compareInt64(a int64, b any, op CompareOp) bool {
switch v := b.(type) {
case int64:
return comparePrimitives(a, v, op)
Expand All @@ -135,7 +135,7 @@ func (p *Parser) compareInt64(a int64, b any, op CompareOp) bool {
}
}

func (p *Parser) compareFloat64(a float64, b any, op CompareOp) bool {
func (p *Parser[K]) compareFloat64(a float64, b any, op CompareOp) bool {
switch v := b.(type) {
case int64:
return comparePrimitives(a, float64(v), op)
Expand All @@ -148,7 +148,7 @@ func (p *Parser) compareFloat64(a float64, b any, op CompareOp) bool {

// a and b are the return values from a Getter; we try to compare them
// according to the given operator.
func (p *Parser) compare(a any, b any, op CompareOp) bool {
func (p *Parser[K]) compare(a any, b any, op CompareOp) bool {
// nils are equal to each other and never equal to anything else,
// so if they're both nil, report equality.
if a == nil && b == nil {
Expand Down
49 changes: 46 additions & 3 deletions pkg/ottl/compare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func Test_compare(t *testing.T) {
for _, tt := range tests {
for _, op := range ops {
t.Run(fmt.Sprintf("%s %v", tt.name, op), func(t *testing.T) {
p := NewParser(nil, nil, nil, componenttest.NewNopTelemetrySettings())
p := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
if got := p.compare(tt.a, tt.b, op); got != tt.want[op] {
t.Errorf("compare(%v, %v, %v) = %v, want %v", tt.a, tt.b, op, got, tt.want[op])
}
Expand All @@ -125,70 +125,113 @@ func Test_compare(t *testing.T) {
}
}

var testParser = NewParser(nil, nil, nil, componenttest.NewNopTelemetrySettings())

// Benchmarks -- these benchmarks compare the performance of comparisons of a variety of data types.
// It's not attempting to be exhaustive, but again, it hits most of the major types and combinations.
// The summary is that they're pretty fast; all the calls to compare are 12 ns/op or less on a 2019 intel
// mac pro laptop, and none of them have any allocations.
func BenchmarkCompareEQInt64(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(i64a, i64b, EQ)
}
}

func BenchmarkCompareEQFloat(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(f64a, f64b, EQ)
}
}

func BenchmarkCompareEQString(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(sa, sb, EQ)
}
}

func BenchmarkCompareEQPString(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(&sa, &sb, EQ)
}
}

func BenchmarkCompareEQBytes(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(ba, bb, EQ)
}
}

func BenchmarkCompareEQNil(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(nil, nil, EQ)
}
}

func BenchmarkCompareNEInt(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(i64a, i64b, NE)
}
}

func BenchmarkCompareNEFloat(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(f64a, f64b, NE)
}
}

func BenchmarkCompareNEString(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(sa, sb, NE)
}
}

func BenchmarkCompareLTFloat(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(f64a, f64b, LT)
}
}

func BenchmarkCompareLTString(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(sa, sb, LT)
}
}

func BenchmarkCompareLTNil(b *testing.B) {
testParser := NewParser[interface{}](nil, nil, nil, componenttest.NewNopTelemetrySettings())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
testParser.compare(nil, nil, LT)
}
Expand Down
Loading

0 comments on commit e4695ee

Please sign in to comment.