diff --git a/client/request/consts.go b/client/request/consts.go index 0e27eaeb3d..18fed52946 100644 --- a/client/request/consts.go +++ b/client/request/consts.go @@ -46,6 +46,8 @@ const ( DeletedFieldName = "_deleted" SumFieldName = "_sum" VersionFieldName = "_version" + MaxFieldName = "_max" + MinFieldName = "_min" // New generated document id from a backed up document, // which might have a different _docID originally. @@ -109,12 +111,16 @@ var ( AverageFieldName: {}, DocIDFieldName: {}, DeletedFieldName: {}, + MaxFieldName: {}, + MinFieldName: {}, } Aggregates = map[string]struct{}{ CountFieldName: {}, SumFieldName: {}, AverageFieldName: {}, + MaxFieldName: {}, + MinFieldName: {}, } CommitQueries = map[string]struct{}{ diff --git a/internal/planner/aggregate.go b/internal/planner/aggregate.go new file mode 100644 index 0000000000..5feb8fb364 --- /dev/null +++ b/internal/planner/aggregate.go @@ -0,0 +1,98 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package planner + +import ( + "github.com/sourcenetwork/defradb/internal/core" + "github.com/sourcenetwork/defradb/internal/planner/mapper" + + "github.com/sourcenetwork/immutable" + "github.com/sourcenetwork/immutable/enumerable" +) + +type number interface { + int64 | float64 +} + +func lessN[T number](a T, b T) bool { + return a < b +} + +func lessO[T number](a immutable.Option[T], b immutable.Option[T]) bool { + if !a.HasValue() { + return true + } + + if !b.HasValue() { + return false + } + + return a.Value() < b.Value() +} + +// inverse returns the logical inverse of the given sort func. +func inverse[T any](original func(T, T) bool) func(T, T) bool { + return func(t1, t2 T) bool { + return !original(t1, t2) + } +} + +// reduces the documents in a slice, skipping over hidden items (a grouping mechanic). +// +// Docs should be reduced with this function to avoid applying offsets twice (once in the +// select, then once here). +func reduceDocs[T any]( + docs []core.Doc, + initialValue T, + reduceFunc func(core.Doc, T) T, +) T { + var value = initialValue + for _, doc := range docs { + if !doc.Hidden { + value = reduceFunc(doc, value) + } + } + return value +} + +func reduceItems[T any, V any]( + source []T, + aggregateTarget *mapper.AggregateTarget, + less func(T, T) bool, + initialValue V, + reduceFunc func(T, V) V, +) (V, error) { + items := enumerable.New(source) + if aggregateTarget.Filter != nil { + items = enumerable.Where(items, func(item T) (bool, error) { + return mapper.RunFilter(item, aggregateTarget.Filter) + }) + } + + if aggregateTarget.OrderBy != nil && len(aggregateTarget.OrderBy.Conditions) > 0 { + if aggregateTarget.OrderBy.Conditions[0].Direction == mapper.ASC { + items = enumerable.Sort(items, less, len(source)) + } else { + items = enumerable.Sort(items, inverse(less), len(source)) + } + } + + if aggregateTarget.Limit != nil { + items = enumerable.Skip(items, aggregateTarget.Limit.Offset) + items = enumerable.Take(items, aggregateTarget.Limit.Limit) + } + + var value = initialValue + err := enumerable.ForEach(items, func(item T) { + value = reduceFunc(item, value) + }) + return value, err +} diff --git a/internal/planner/explain.go b/internal/planner/explain.go index 76679a85e3..860ea39df1 100644 --- a/internal/planner/explain.go +++ b/internal/planner/explain.go @@ -39,6 +39,8 @@ var ( _ explainablePlanNode = (*deleteNode)(nil) _ explainablePlanNode = (*groupNode)(nil) _ explainablePlanNode = (*limitNode)(nil) + _ explainablePlanNode = (*maxNode)(nil) + _ explainablePlanNode = (*minNode)(nil) _ explainablePlanNode = (*orderNode)(nil) _ explainablePlanNode = (*scanNode)(nil) _ explainablePlanNode = (*selectNode)(nil) diff --git a/internal/planner/max.go b/internal/planner/max.go new file mode 100644 index 0000000000..dbcc991268 --- /dev/null +++ b/internal/planner/max.go @@ -0,0 +1,255 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package planner + +import ( + "math/big" + + "github.com/sourcenetwork/immutable" + + "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/internal/core" + "github.com/sourcenetwork/defradb/internal/planner/mapper" +) + +type maxNode struct { + documentIterator + docMapper + + p *Planner + plan planNode + parent *mapper.Select + + // virtualFieldIndex is the index of the field + // that contains the result of the aggregate. + virtualFieldIndex int + aggregateMapping []mapper.AggregateTarget + + execInfo maxExecInfo +} + +type maxExecInfo struct { + // Total number of times maxNode was executed. + iterations uint64 +} + +func (p *Planner) Max( + field *mapper.Aggregate, + parent *mapper.Select, +) (*maxNode, error) { + return &maxNode{ + p: p, + parent: parent, + aggregateMapping: field.AggregateTargets, + virtualFieldIndex: field.Index, + docMapper: docMapper{field.DocumentMapping}, + }, nil +} + +func (n *maxNode) Kind() string { return "maxNode" } +func (n *maxNode) Init() error { return n.plan.Init() } +func (n *maxNode) Start() error { return n.plan.Start() } +func (n *maxNode) Spans(spans core.Spans) { n.plan.Spans(spans) } +func (n *maxNode) Close() error { return n.plan.Close() } +func (n *maxNode) Source() planNode { return n.plan } +func (n *maxNode) SetPlan(p planNode) { n.plan = p } + +func (n *maxNode) simpleExplain() (map[string]any, error) { + sourceExplanations := make([]map[string]any, len(n.aggregateMapping)) + + for i, source := range n.aggregateMapping { + simpleExplainMap := map[string]any{} + + // Add the filter attribute if it exists. + if source.Filter == nil { + simpleExplainMap[filterLabel] = nil + } else { + // get the target aggregate document mapping. Since the filters + // are relative to the target aggregate collection (and doc mapper). + var targetMap *core.DocumentMapping + if source.Index < len(n.documentMapping.ChildMappings) && + n.documentMapping.ChildMappings[source.Index] != nil { + targetMap = n.documentMapping.ChildMappings[source.Index] + } else { + targetMap = n.documentMapping + } + simpleExplainMap[filterLabel] = source.Filter.ToMap(targetMap) + } + + // Add the main field name. + simpleExplainMap[fieldNameLabel] = source.Field.Name + + // Add the child field name if it exists. + if source.ChildTarget.HasValue { + simpleExplainMap[childFieldNameLabel] = source.ChildTarget.Name + } else { + simpleExplainMap[childFieldNameLabel] = nil + } + + sourceExplanations[i] = simpleExplainMap + } + + return map[string]any{ + sourcesLabel: sourceExplanations, + }, nil +} + +// Explain method returns a map containing all attributes of this node that +// are to be explained, subscribes / opts-in this node to be an explainablePlanNode. +func (n *maxNode) Explain(explainType request.ExplainType) (map[string]any, error) { + switch explainType { + case request.SimpleExplain: + return n.simpleExplain() + + case request.ExecuteExplain: + return map[string]any{ + "iterations": n.execInfo.iterations, + }, nil + + default: + return nil, ErrUnknownExplainRequestType + } +} + +func (n *maxNode) Next() (bool, error) { + n.execInfo.iterations++ + + hasNext, err := n.plan.Next() + if err != nil || !hasNext { + return hasNext, err + } + n.currentValue = n.plan.Value() + + var max *big.Float + isFloat := false + + for _, source := range n.aggregateMapping { + child := n.currentValue.Fields[source.Index] + var collectionMax *big.Float + var err error + switch childCollection := child.(type) { + case []core.Doc: + collectionMax = reduceDocs( + childCollection, + nil, + func(childItem core.Doc, value *big.Float) *big.Float { + childProperty := childItem.Fields[source.ChildTarget.Index] + res := &big.Float{} + switch v := childProperty.(type) { + case int: + res = res.SetInt64(int64(v)) + case int64: + res = res.SetInt64(v) + case uint64: + res = res.SetUint64(v) + case float64: + res = res.SetFloat64(v) + default: + return nil + } + if value == nil || res.Cmp(value) > 0 { + return res + } + return value + }, + ) + + case []int64: + collectionMax, err = reduceItems( + childCollection, + &source, + lessN[int64], + nil, + func(childItem int64, value *big.Float) *big.Float { + res := (&big.Float{}).SetInt64(childItem) + if value == nil || res.Cmp(value) > 0 { + return res + } + return value + }, + ) + + case []immutable.Option[int64]: + collectionMax, err = reduceItems( + childCollection, + &source, + lessO[int64], + nil, + func(childItem immutable.Option[int64], value *big.Float) *big.Float { + if !childItem.HasValue() { + return value + } + res := (&big.Float{}).SetInt64(childItem.Value()) + if value == nil || res.Cmp(value) > 0 { + return res + } + return value + }, + ) + + case []float64: + collectionMax, err = reduceItems( + childCollection, + &source, + lessN[float64], + nil, + func(childItem float64, value *big.Float) *big.Float { + res := big.NewFloat(childItem) + if value == nil || res.Cmp(value) > 0 { + return res + } + return value + }, + ) + + case []immutable.Option[float64]: + collectionMax, err = reduceItems( + childCollection, + &source, + lessO[float64], + nil, + func(childItem immutable.Option[float64], value *big.Float) *big.Float { + if !childItem.HasValue() { + return value + } + res := big.NewFloat(childItem.Value()) + if value == nil || res.Cmp(value) > 0 { + return res + } + return value + }, + ) + } + if err != nil { + return false, err + } + if collectionMax == nil || (max != nil && collectionMax.Cmp(max) <= 0) { + continue + } + isTargetFloat, err := n.p.isValueFloat(n.parent, &source) + if err != nil { + return false, err + } + isFloat = isTargetFloat + max = collectionMax + } + + if max == nil { + n.currentValue.Fields[n.virtualFieldIndex] = nil + } else if isFloat { + res, _ := max.Float64() + n.currentValue.Fields[n.virtualFieldIndex] = res + } else { + res, _ := max.Int64() + n.currentValue.Fields[n.virtualFieldIndex] = res + } + return true, nil +} diff --git a/internal/planner/min.go b/internal/planner/min.go new file mode 100644 index 0000000000..9be8ecd30a --- /dev/null +++ b/internal/planner/min.go @@ -0,0 +1,255 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package planner + +import ( + "math/big" + + "github.com/sourcenetwork/immutable" + + "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/internal/core" + "github.com/sourcenetwork/defradb/internal/planner/mapper" +) + +type minNode struct { + documentIterator + docMapper + + p *Planner + plan planNode + parent *mapper.Select + + // virtualFieldIndex is the index of the field + // that contains the result of the aggregate. + virtualFieldIndex int + aggregateMapping []mapper.AggregateTarget + + execInfo minExecInfo +} + +type minExecInfo struct { + // Total number of times minNode was executed. + iterations uint64 +} + +func (p *Planner) Min( + field *mapper.Aggregate, + parent *mapper.Select, +) (*minNode, error) { + return &minNode{ + p: p, + parent: parent, + aggregateMapping: field.AggregateTargets, + virtualFieldIndex: field.Index, + docMapper: docMapper{field.DocumentMapping}, + }, nil +} + +func (n *minNode) Kind() string { return "minNode" } +func (n *minNode) Init() error { return n.plan.Init() } +func (n *minNode) Start() error { return n.plan.Start() } +func (n *minNode) Spans(spans core.Spans) { n.plan.Spans(spans) } +func (n *minNode) Close() error { return n.plan.Close() } +func (n *minNode) Source() planNode { return n.plan } +func (n *minNode) SetPlan(p planNode) { n.plan = p } + +func (n *minNode) simpleExplain() (map[string]any, error) { + sourceExplanations := make([]map[string]any, len(n.aggregateMapping)) + + for i, source := range n.aggregateMapping { + simpleExplainMap := map[string]any{} + + // Add the filter attribute if it exists. + if source.Filter == nil { + simpleExplainMap[filterLabel] = nil + } else { + // get the target aggregate document mapping. Since the filters + // are relative to the target aggregate collection (and doc mapper). + var targetMap *core.DocumentMapping + if source.Index < len(n.documentMapping.ChildMappings) && + n.documentMapping.ChildMappings[source.Index] != nil { + targetMap = n.documentMapping.ChildMappings[source.Index] + } else { + targetMap = n.documentMapping + } + simpleExplainMap[filterLabel] = source.Filter.ToMap(targetMap) + } + + // Add the main field name. + simpleExplainMap[fieldNameLabel] = source.Field.Name + + // Add the child field name if it exists. + if source.ChildTarget.HasValue { + simpleExplainMap[childFieldNameLabel] = source.ChildTarget.Name + } else { + simpleExplainMap[childFieldNameLabel] = nil + } + + sourceExplanations[i] = simpleExplainMap + } + + return map[string]any{ + sourcesLabel: sourceExplanations, + }, nil +} + +// Explain method returns a map containing all attributes of this node that +// are to be explained, subscribes / opts-in this node to be an explainablePlanNode. +func (n *minNode) Explain(explainType request.ExplainType) (map[string]any, error) { + switch explainType { + case request.SimpleExplain: + return n.simpleExplain() + + case request.ExecuteExplain: + return map[string]any{ + "iterations": n.execInfo.iterations, + }, nil + + default: + return nil, ErrUnknownExplainRequestType + } +} + +func (n *minNode) Next() (bool, error) { + n.execInfo.iterations++ + + hasNext, err := n.plan.Next() + if err != nil || !hasNext { + return hasNext, err + } + n.currentValue = n.plan.Value() + + var min *big.Float + isFloat := false + + for _, source := range n.aggregateMapping { + child := n.currentValue.Fields[source.Index] + var collectionMin *big.Float + var err error + switch childCollection := child.(type) { + case []core.Doc: + collectionMin = reduceDocs( + childCollection, + nil, + func(childItem core.Doc, value *big.Float) *big.Float { + childProperty := childItem.Fields[source.ChildTarget.Index] + res := &big.Float{} + switch v := childProperty.(type) { + case int: + res = res.SetInt64(int64(v)) + case int64: + res = res.SetInt64(v) + case uint64: + res = res.SetUint64(v) + case float64: + res = res.SetFloat64(v) + default: + return nil + } + if value == nil || res.Cmp(value) < 0 { + return res + } + return value + }, + ) + + case []int64: + collectionMin, err = reduceItems( + childCollection, + &source, + lessN[int64], + nil, + func(childItem int64, value *big.Float) *big.Float { + res := (&big.Float{}).SetInt64(childItem) + if value == nil || res.Cmp(value) < 0 { + return res + } + return value + }, + ) + + case []immutable.Option[int64]: + collectionMin, err = reduceItems( + childCollection, + &source, + lessO[int64], + nil, + func(childItem immutable.Option[int64], value *big.Float) *big.Float { + if !childItem.HasValue() { + return value + } + res := (&big.Float{}).SetInt64(childItem.Value()) + if value == nil || res.Cmp(value) < 0 { + return res + } + return value + }, + ) + + case []float64: + collectionMin, err = reduceItems( + childCollection, + &source, + lessN[float64], + nil, + func(childItem float64, value *big.Float) *big.Float { + res := big.NewFloat(childItem) + if value == nil || res.Cmp(value) < 0 { + return res + } + return value + }, + ) + + case []immutable.Option[float64]: + collectionMin, err = reduceItems( + childCollection, + &source, + lessO[float64], + nil, + func(childItem immutable.Option[float64], value *big.Float) *big.Float { + if !childItem.HasValue() { + return value + } + res := big.NewFloat(childItem.Value()) + if value == nil || res.Cmp(value) < 0 { + return res + } + return value + }, + ) + } + if err != nil { + return false, err + } + if collectionMin == nil || (min != nil && collectionMin.Cmp(min) >= 0) { + continue + } + isTargetFloat, err := n.p.isValueFloat(n.parent, &source) + if err != nil { + return false, err + } + isFloat = isTargetFloat + min = collectionMin + } + + if min == nil { + n.currentValue.Fields[n.virtualFieldIndex] = nil + } else if isFloat { + res, _ := min.Float64() + n.currentValue.Fields[n.virtualFieldIndex] = res + } else { + res, _ := min.Int64() + n.currentValue.Fields[n.virtualFieldIndex] = res + } + return true, nil +} diff --git a/internal/planner/operations.go b/internal/planner/operations.go index 6cbf7c24d4..73fe1450bb 100644 --- a/internal/planner/operations.go +++ b/internal/planner/operations.go @@ -18,6 +18,8 @@ var ( _ planNode = (*deleteNode)(nil) _ planNode = (*groupNode)(nil) _ planNode = (*limitNode)(nil) + _ planNode = (*maxNode)(nil) + _ planNode = (*minNode)(nil) _ planNode = (*multiScanNode)(nil) _ planNode = (*orderNode)(nil) _ planNode = (*parallelNode)(nil) diff --git a/internal/planner/select.go b/internal/planner/select.go index 3c5cc58bee..064f9b2fec 100644 --- a/internal/planner/select.go +++ b/internal/planner/select.go @@ -347,6 +347,10 @@ func (n *selectNode) initFields(selectReq *mapper.Select) ([]aggregateNode, erro plan, aggregateError = n.planner.Sum(f, selectReq) case request.AverageFieldName: plan, aggregateError = n.planner.Average(f) + case request.MaxFieldName: + plan, aggregateError = n.planner.Max(f, selectReq) + case request.MinFieldName: + plan, aggregateError = n.planner.Min(f, selectReq) } if aggregateError != nil { diff --git a/internal/planner/sum.go b/internal/planner/sum.go index ff0b714ebf..177dd72e3a 100644 --- a/internal/planner/sum.go +++ b/internal/planner/sum.go @@ -12,7 +12,6 @@ package planner import ( "github.com/sourcenetwork/immutable" - "github.com/sourcenetwork/immutable/enumerable" "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/client/request" @@ -231,65 +230,69 @@ func (n *sumNode) Next() (bool, error) { var err error switch childCollection := child.(type) { case []core.Doc: - collectionSum = sumDocs(childCollection, func(childItem core.Doc) float64 { + collectionSum = reduceDocs(childCollection, 0, func(childItem core.Doc, value float64) float64 { childProperty := childItem.Fields[source.ChildTarget.Index] switch v := childProperty.(type) { case int: - return float64(v) + return value + float64(v) case int64: - return float64(v) + return value + float64(v) case uint64: - return float64(v) + return value + float64(v) case float64: - return v + return value + v default: // return nothing, cannot be summed - return 0 + return value + 0 } }) case []int64: - collectionSum, err = sumItems( + collectionSum, err = reduceItems( childCollection, &source, lessN[int64], - func(childItem int64) float64 { - return float64(childItem) + 0, + func(childItem int64, value float64) float64 { + return value + float64(childItem) }, ) case []immutable.Option[int64]: - collectionSum, err = sumItems( + collectionSum, err = reduceItems( childCollection, &source, lessO[int64], - func(childItem immutable.Option[int64]) float64 { + 0, + func(childItem immutable.Option[int64], value float64) float64 { if !childItem.HasValue() { - return 0 + return value + 0 } - return float64(childItem.Value()) + return value + float64(childItem.Value()) }, ) case []float64: - collectionSum, err = sumItems( + collectionSum, err = reduceItems( childCollection, &source, lessN[float64], - func(childItem float64) float64 { - return childItem + 0, + func(childItem float64, value float64) float64 { + return value + childItem }, ) case []immutable.Option[float64]: - collectionSum, err = sumItems( + collectionSum, err = reduceItems( childCollection, &source, lessO[float64], - func(childItem immutable.Option[float64]) float64 { + 0, + func(childItem immutable.Option[float64], value float64) float64 { if !childItem.HasValue() { - return 0 + return value + 0 } - return childItem.Value() + return value + childItem.Value() }, ) } @@ -310,78 +313,4 @@ func (n *sumNode) Next() (bool, error) { return true, nil } -// offsets sums the documents in a slice, skipping over hidden items (a grouping mechanic). -// Docs should be counted with this function to avoid applying offsets twice (once in the -// select, then once here). -func sumDocs(docs []core.Doc, toFloat func(core.Doc) float64) float64 { - var sum float64 = 0 - for _, doc := range docs { - if !doc.Hidden { - sum += toFloat(doc) - } - } - - return sum -} - -func sumItems[T any]( - source []T, - aggregateTarget *mapper.AggregateTarget, - less func(T, T) bool, - toFloat func(T) float64, -) (float64, error) { - items := enumerable.New(source) - if aggregateTarget.Filter != nil { - items = enumerable.Where(items, func(item T) (bool, error) { - return mapper.RunFilter(item, aggregateTarget.Filter) - }) - } - - if aggregateTarget.OrderBy != nil && len(aggregateTarget.OrderBy.Conditions) > 0 { - if aggregateTarget.OrderBy.Conditions[0].Direction == mapper.ASC { - items = enumerable.Sort(items, less, len(source)) - } else { - items = enumerable.Sort(items, reverse(less), len(source)) - } - } - - if aggregateTarget.Limit != nil { - items = enumerable.Skip(items, aggregateTarget.Limit.Offset) - items = enumerable.Take(items, aggregateTarget.Limit.Limit) - } - - var sum float64 = 0 - err := enumerable.ForEach(items, func(item T) { - sum += toFloat(item) - }) - - return sum, err -} - func (n *sumNode) SetPlan(p planNode) { n.plan = p } - -type number interface { - int64 | float64 -} - -func lessN[T number](a T, b T) bool { - return a < b -} - -func lessO[T number](a immutable.Option[T], b immutable.Option[T]) bool { - if !a.HasValue() { - return true - } - - if !b.HasValue() { - return false - } - - return a.Value() < b.Value() -} - -func reverse[T any](original func(T, T) bool) func(T, T) bool { - return func(t1, t2 T) bool { - return !original(t1, t2) - } -} diff --git a/internal/planner/top.go b/internal/planner/top.go index d5faa491c1..ce2ce4e6dc 100644 --- a/internal/planner/top.go +++ b/internal/planner/top.go @@ -203,6 +203,10 @@ func (p *Planner) Top(m *mapper.Select) (*topLevelNode, error) { child, err = p.Sum(f, m) case request.AverageFieldName: child, err = p.Average(f) + case request.MaxFieldName: + child, err = p.Max(f, m) + case request.MinFieldName: + child, err = p.Min(f, m) } if err != nil { return nil, err diff --git a/internal/request/graphql/schema/generate.go b/internal/request/graphql/schema/generate.go index 85491f5ee1..f326a8232a 100644 --- a/internal/request/graphql/schema/generate.go +++ b/internal/request/graphql/schema/generate.go @@ -651,10 +651,21 @@ func (g *Generator) genAggregateFields() error { return err } t.AddFieldConfig(averageField.Name, &averageField) + + maxField, err := g.genMaximumFieldConfig(t) + if err != nil { + return err + } + t.AddFieldConfig(maxField.Name, &maxField) + + minField, err := g.genMinimumFieldConfig(t) + if err != nil { + return err + } + t.AddFieldConfig(minField.Name, &minField) } queryType := g.manager.schema.QueryType() - topLevelCountField := genTopLevelCount(topLevelCountInputs) queryType.AddFieldConfig(topLevelCountField.Name, topLevelCountField) @@ -695,12 +706,33 @@ func genTopLevelNumericAggregates(topLevelNumericAggInputs map[string]*gql.Input Args: gql.FieldConfigArgument{}, } + topLevelMaximumField := gql.Field{ + Name: request.MaxFieldName, + Description: schemaTypes.MaximumFieldDescription, + Type: gql.Float, + Args: gql.FieldConfigArgument{}, + } + + topLevelMinimumField := gql.Field{ + Name: request.MinFieldName, + Description: schemaTypes.MinimumFieldDescription, + Type: gql.Float, + Args: gql.FieldConfigArgument{}, + } + for name, inputObject := range topLevelNumericAggInputs { topLevelSumField.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) topLevelAverageField.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) + topLevelMaximumField.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) + topLevelMinimumField.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) } - return []*gql.Field{&topLevelSumField, &topLevelAverageField} + return []*gql.Field{ + &topLevelSumField, + &topLevelAverageField, + &topLevelMaximumField, + &topLevelMinimumField, + } } func (g *Generator) genCountFieldConfig(obj *gql.Object) (gql.Field, error) { @@ -741,50 +773,68 @@ func (g *Generator) genCountFieldConfig(obj *gql.Object) (gql.Field, error) { } func (g *Generator) genSumFieldConfig(obj *gql.Object) (gql.Field, error) { - childTypesByFieldName := map[string]gql.Type{} + field := gql.Field{ + Name: request.SumFieldName, + Description: schemaTypes.SumFieldDescription, + Type: gql.Float, + Args: gql.FieldConfigArgument{}, + } - for _, field := range obj.Fields() { - // we can only sum list items - listType, isList := field.Type.(*gql.List) - if !isList { - continue - } + childTypesByFieldName := g.getNumericFields(obj) + for name, inputObject := range childTypesByFieldName { + field.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) + } + return field, nil +} - var inputObjectName string - if isNumericArray(listType) { - inputObjectName = genNumericInlineArraySelectorName(obj.Name(), field.Name) - } else { - inputObjectName = genNumericObjectSelectorName(listType.OfType.Name()) - } +func (g *Generator) genMinimumFieldConfig(obj *gql.Object) (gql.Field, error) { + field := gql.Field{ + Name: request.MinFieldName, + Description: schemaTypes.MinimumFieldDescription, + Type: gql.Float, + Args: gql.FieldConfigArgument{}, + } - subSumType, isSubTypeSumable := g.manager.schema.TypeMap()[inputObjectName] - // If the item is not in the type map, it must contain no summable - // fields (e.g. no Int/Floats) - if !isSubTypeSumable { - continue - } - childTypesByFieldName[field.Name] = subSumType + childTypesByFieldName := g.getNumericFields(obj) + for name, inputObject := range childTypesByFieldName { + field.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) } + return field, nil +} +func (g *Generator) genMaximumFieldConfig(obj *gql.Object) (gql.Field, error) { field := gql.Field{ - Name: request.SumFieldName, - Description: schemaTypes.SumFieldDescription, + Name: request.MaxFieldName, + Description: schemaTypes.MaximumFieldDescription, Type: gql.Float, Args: gql.FieldConfigArgument{}, } + childTypesByFieldName := g.getNumericFields(obj) for name, inputObject := range childTypesByFieldName { field.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) } - return field, nil } func (g *Generator) genAverageFieldConfig(obj *gql.Object) (gql.Field, error) { - childTypesByFieldName := map[string]gql.Type{} + field := gql.Field{ + Name: request.AverageFieldName, + Description: schemaTypes.AverageFieldDescription, + Type: gql.Float, + Args: gql.FieldConfigArgument{}, + } + + childTypesByFieldName := g.getNumericFields(obj) + for name, inputObject := range childTypesByFieldName { + field.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) + } + return field, nil +} +func (g *Generator) getNumericFields(obj *gql.Object) map[string]gql.Type { + fieldTypes := map[string]gql.Type{} for _, field := range obj.Fields() { - // we can only sum list items listType, isList := field.Type.(*gql.List) if !isList { continue @@ -798,26 +848,12 @@ func (g *Generator) genAverageFieldConfig(obj *gql.Object) (gql.Field, error) { } subAverageType, isSubTypeAveragable := g.manager.schema.TypeMap()[inputObjectName] - // If the item is not in the type map, it must contain no averagable - // fields (e.g. no Int/Floats) if !isSubTypeAveragable { continue } - childTypesByFieldName[field.Name] = subAverageType - } - - field := gql.Field{ - Name: request.AverageFieldName, - Description: schemaTypes.AverageFieldDescription, - Type: gql.Float, - Args: gql.FieldConfigArgument{}, - } - - for name, inputObject := range childTypesByFieldName { - field.Args[name] = schemaTypes.NewArgConfig(inputObject, inputObject.Description()) + fieldTypes[field.Name] = subAverageType } - - return field, nil + return fieldTypes } func (g *Generator) genNumericInlineArraySelectorObject(obj *gql.Object) []*gql.InputObject { @@ -953,6 +989,8 @@ func (g *Generator) genNumericAggregateBaseArgInputs(obj *gql.Object) *gql.Input // A child aggregate will always be aggregatable, as it can be present via an inner grouping fieldsEnumCfg.Values[request.SumFieldName] = &gql.EnumValueConfig{Value: request.SumFieldName} fieldsEnumCfg.Values[request.AverageFieldName] = &gql.EnumValueConfig{Value: request.AverageFieldName} + fieldsEnumCfg.Values[request.MinFieldName] = &gql.EnumValueConfig{Value: request.MinFieldName} + fieldsEnumCfg.Values[request.MaxFieldName] = &gql.EnumValueConfig{Value: request.MaxFieldName} if !hasSumableFields { return nil, nil diff --git a/internal/request/graphql/schema/types/descriptions.go b/internal/request/graphql/schema/types/descriptions.go index 213266d891..e442545995 100644 --- a/internal/request/graphql/schema/types/descriptions.go +++ b/internal/request/graphql/schema/types/descriptions.go @@ -124,6 +124,16 @@ Returns the total sum of the specified field values within the specified child s Returns the average of the specified field values within the specified child sets. If multiple fields/sets are specified, the combined average of all items within each set (true average, not an average of averages) will be returned as a single value. +` + MaximumFieldDescription string = ` +Returns the maximum of the specified field values within the specified child sets. If + multiple fields/sets are specified, the combined maximum of all items within each set + will be returned as a single value. +` + MinimumFieldDescription string = ` +Returns the minimum of the specified field values within the specified child sets. If + multiple fields/sets are specified, the combined minimum of all items within each set + will be returned as a single value. ` booleanOperatorBlockDescription string = ` These are the set of filter operators available for use when filtering on Boolean diff --git a/tests/integration/explain.go b/tests/integration/explain.go index 325e69b3f7..4cdebe9103 100644 --- a/tests/integration/explain.go +++ b/tests/integration/explain.go @@ -41,6 +41,8 @@ var ( "deleteNode": {}, "groupNode": {}, "limitNode": {}, + "maxNode": {}, + "minNode": {}, "multiScanNode": {}, "orderNode": {}, "parallelNode": {}, diff --git a/tests/integration/explain/debug/top_with_max_test.go b/tests/integration/explain/debug/top_with_max_test.go new file mode 100644 index 0000000000..b79f469511 --- /dev/null +++ b/tests/integration/explain/debug/top_with_max_test.go @@ -0,0 +1,96 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var topLevelMaxPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "topLevelNode": []dataMap{ + { + "selectTopNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + { + "maxNode": dataMap{}, + }, + }, + }, + }, + }, +} + +func TestDebugExplain_TopLevelMaxRequest_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) top-level max request.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + _max( + Author: { + field: age + } + ) + }`, + + ExpectedPatterns: topLevelMaxPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplain_TopLevelMaxRequestWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) top-level max request with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + _max( + Author: { + field: age, + filter: { + age: { + _gt: 26 + } + } + } + ) + }`, + + ExpectedPatterns: topLevelMaxPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/debug/top_with_min_test.go b/tests/integration/explain/debug/top_with_min_test.go new file mode 100644 index 0000000000..1504f23905 --- /dev/null +++ b/tests/integration/explain/debug/top_with_min_test.go @@ -0,0 +1,96 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var topLevelMinPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "topLevelNode": []dataMap{ + { + "selectTopNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + { + "minNode": dataMap{}, + }, + }, + }, + }, + }, +} + +func TestDebugExplain_TopLevelMinRequest_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) top-level min request.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + _min( + Author: { + field: age + } + ) + }`, + + ExpectedPatterns: topLevelMinPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplain_TopLevelMinRequestWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) top-level min request with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + _min( + Author: { + field: age, + filter: { + age: { + _gt: 26 + } + } + } + ) + }`, + + ExpectedPatterns: topLevelMinPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/debug/with_max_join_test.go b/tests/integration/explain/debug/with_max_join_test.go new file mode 100644 index 0000000000..fd1bf25bfb --- /dev/null +++ b/tests/integration/explain/debug/with_max_join_test.go @@ -0,0 +1,151 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var debugMaxTypeIndexJoinManyPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "typeIndexJoin": dataMap{ + "typeJoinMany": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, +} + +func TestDebugExplainRequest_WithMaxOnOneToManyJoinedField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with max on a one-to-many joined field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + _docID + MaxPages: _max( + books: {field: pages} + ) + } + }`, + + ExpectedPatterns: debugMaxTypeIndexJoinManyPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplainRequest_WithMaxOnOneToManyJoinedFieldWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with max on a one-to-many joined field, with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + MaxPages: _max( + articles: { + field: pages, + filter: { + name: { + _eq: "To my dear readers" + } + } + } + ) + } + }`, + + ExpectedPatterns: debugMaxTypeIndexJoinManyPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplainRequest_WithMaxOnOneToManyJoinedFieldWithManySources_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with max on a one-to-many joined field with many sources.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + MaxPages: _max( + books: {field: pages}, + articles: {field: pages} + ) + } + }`, + + ExpectedPatterns: dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "parallelNode": []dataMap{ + { + "typeIndexJoin": dataMap{ + "typeJoinMany": debugTypeJoinPattern, + }, + }, + { + "typeIndexJoin": dataMap{ + "typeJoinMany": debugTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/debug/with_max_test.go b/tests/integration/explain/debug/with_max_test.go new file mode 100644 index 0000000000..862056b3a7 --- /dev/null +++ b/tests/integration/explain/debug/with_max_test.go @@ -0,0 +1,59 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var maxPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + }, + }, + }, +} + +func TestDebugExplainRequestWithMaxOnInlineArrayField_ChildFieldWillBeEmpty(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with max on an inline array field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Book { + name + MaxChapterPages: _max(chapterPages: {}) + } + }`, + + ExpectedPatterns: maxPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/debug/with_min_join_test.go b/tests/integration/explain/debug/with_min_join_test.go new file mode 100644 index 0000000000..afbecdc687 --- /dev/null +++ b/tests/integration/explain/debug/with_min_join_test.go @@ -0,0 +1,151 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var debugMinTypeIndexJoinManyPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "typeIndexJoin": dataMap{ + "typeJoinMany": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, +} + +func TestDebugExplainRequest_WithMinOnOneToManyJoinedField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with min on a one-to-many joined field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + _docID + MinPages: _min( + books: {field: pages} + ) + } + }`, + + ExpectedPatterns: debugMinTypeIndexJoinManyPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplainRequest_WithMinOnOneToManyJoinedFieldWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with min on a one-to-many joined field, with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + MinPages: _min( + articles: { + field: pages, + filter: { + name: { + _eq: "To my dear readers" + } + } + } + ) + } + }`, + + ExpectedPatterns: debugMinTypeIndexJoinManyPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDebugExplainRequest_WithMinOnOneToManyJoinedFieldWithManySources_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with min on a one-to-many joined field with many sources.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Author { + name + MinPages: _min( + books: {field: pages}, + articles: {field: pages} + ) + } + }`, + + ExpectedPatterns: dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "parallelNode": []dataMap{ + { + "typeIndexJoin": dataMap{ + "typeJoinMany": debugTypeJoinPattern, + }, + }, + { + "typeIndexJoin": dataMap{ + "typeJoinMany": debugTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/debug/with_min_test.go b/tests/integration/explain/debug/with_min_test.go new file mode 100644 index 0000000000..28319dc648 --- /dev/null +++ b/tests/integration/explain/debug/with_min_test.go @@ -0,0 +1,59 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_debug + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var minPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + }, + }, + }, +} + +func TestDebugExplainRequestWithMinOnInlineArrayField_ChildFieldWillBeEmpty(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (debug) request with min on an inline array field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain(type: debug) { + Book { + name + MinChapterPages: _min(chapterPages: {}) + } + }`, + + ExpectedPatterns: minPattern, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/top_with_max_test.go b/tests/integration/explain/default/top_with_max_test.go new file mode 100644 index 0000000000..16d53b5007 --- /dev/null +++ b/tests/integration/explain/default/top_with_max_test.go @@ -0,0 +1,166 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var topLevelMaxPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "topLevelNode": []dataMap{ + { + "selectTopNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + { + "maxNode": dataMap{}, + }, + }, + }, + }, + }, +} + +func TestDefaultExplain_WithTopLevelMaxRequest_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) top-level max request.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + _max( + Author: { + field: age + } + ) + }`, + + ExpectedPatterns: topLevelMaxPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "maxNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "Author", + "childFieldName": "age", + "filter": nil, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplain_WithTopLevelMaxRequestWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) top-level max request with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + _max( + Author: { + field: age, + filter: { + age: { + _gt: 26 + } + } + } + ) + }`, + + ExpectedPatterns: topLevelMaxPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": dataMap{ + "age": dataMap{ + "_gt": int32(26), + }, + }, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "maxNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "Author", + "childFieldName": "age", + "filter": dataMap{ + "age": dataMap{ + "_gt": int32(26), + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/top_with_min_test.go b/tests/integration/explain/default/top_with_min_test.go new file mode 100644 index 0000000000..b212953a60 --- /dev/null +++ b/tests/integration/explain/default/top_with_min_test.go @@ -0,0 +1,166 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var topLevelMinPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "topLevelNode": []dataMap{ + { + "selectTopNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + { + "minNode": dataMap{}, + }, + }, + }, + }, + }, +} + +func TestDefaultExplain_WithTopLevelMinRequest_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) top-level min request.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + _min( + Author: { + field: age + } + ) + }`, + + ExpectedPatterns: topLevelMinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "minNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "Author", + "childFieldName": "age", + "filter": nil, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplain_WithTopLevelMinRequestWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) top-level min request with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + _min( + Author: { + field: age, + filter: { + age: { + _gt: 26 + } + } + } + ) + }`, + + ExpectedPatterns: topLevelMinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": dataMap{ + "age": dataMap{ + "_gt": int32(26), + }, + }, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "minNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "Author", + "childFieldName": "age", + "filter": dataMap{ + "age": dataMap{ + "_gt": int32(26), + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/with_max_join_test.go b/tests/integration/explain/default/with_max_join_test.go new file mode 100644 index 0000000000..a282f9f134 --- /dev/null +++ b/tests/integration/explain/default/with_max_join_test.go @@ -0,0 +1,375 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + "github.com/sourcenetwork/immutable" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var maxTypeIndexJoinPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "typeIndexJoin": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, +} + +func TestDefaultExplainRequest_WithMaxOnOneToManyJoinedField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with max on a one-to-many joined field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + _docID + TotalPages: _max( + books: {field: pages} + ) + } + }`, + + ExpectedPatterns: maxTypeIndexJoinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "maxNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "books", + "childFieldName": "pages", + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "books", + }, + }, + { + TargetNodeName: "scanNode", // inside of root + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplainRequest_WithMaxOnOneToManyJoinedFieldWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with max on a one-to-many joined field, with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + TotalPages: _max( + articles: { + field: pages, + filter: { + name: { + _eq: "To my dear readers" + } + } + } + ) + } + }`, + + ExpectedPatterns: maxTypeIndexJoinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "maxNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "articles", + "childFieldName": "pages", + "filter": dataMap{ + "name": dataMap{ + "_eq": "To my dear readers", + }, + }, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "articles", + }, + }, + { + TargetNodeName: "scanNode", // inside of root + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "1", + "collectionName": "Article", + "filter": dataMap{ + "name": dataMap{ + "_eq": "To my dear readers", + }, + }, + "spans": []dataMap{ + { + "start": "/1", + "end": "/2", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplainRequest_WithMaxOnOneToManyJoinedFieldWithManySources_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with max on a one-to-many joined field with many sources.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + TotalPages: _max( + books: {field: pages}, + articles: {field: pages} + ) + } + }`, + + ExpectedPatterns: dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "parallelNode": []dataMap{ + { + "typeIndexJoin": normalTypeJoinPattern, + }, + { + "typeIndexJoin": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, + }, + }, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "maxNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "childFieldName": "pages", + "fieldName": "books", + "filter": nil, + }, + + { + "childFieldName": "pages", + "fieldName": "articles", + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + OccurancesToSkip: 0, + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "books", + }, + }, + { + TargetNodeName: "scanNode", // inside of 1st root type + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of 1st subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + OccurancesToSkip: 1, + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "articles", + }, + }, + { + TargetNodeName: "scanNode", // inside of 2nd root type (AKA: subType's root) + OccurancesToSkip: 2, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of 2nd subType (AKA: subType's subtype) + OccurancesToSkip: 3, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "1", + "collectionName": "Article", + "filter": nil, + "spans": []dataMap{ + { + "start": "/1", + "end": "/2", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/with_max_test.go b/tests/integration/explain/default/with_max_test.go new file mode 100644 index 0000000000..823e3d5def --- /dev/null +++ b/tests/integration/explain/default/with_max_test.go @@ -0,0 +1,90 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var maxPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + }, + }, + }, +} + +func TestDefaultExplainRequest_WithMaxOnInlineArrayField_ChildFieldWillBeEmpty(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with max on an inline array field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Book { + name + MaxChapterPages: _max(chapterPages: {}) + } + }`, + + ExpectedPatterns: maxPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "maxNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "chapterPages", + "childFieldName": nil, + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/with_min_join_test.go b/tests/integration/explain/default/with_min_join_test.go new file mode 100644 index 0000000000..2e12bf1788 --- /dev/null +++ b/tests/integration/explain/default/with_min_join_test.go @@ -0,0 +1,375 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + "github.com/sourcenetwork/immutable" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var minTypeIndexJoinPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "typeIndexJoin": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, +} + +func TestDefaultExplainRequest_WithMinOnOneToManyJoinedField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with min on a one-to-many joined field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + _docID + TotalPages: _min( + books: {field: pages} + ) + } + }`, + + ExpectedPatterns: minTypeIndexJoinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "minNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "books", + "childFieldName": "pages", + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "books", + }, + }, + { + TargetNodeName: "scanNode", // inside of root + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplainRequest_WithMinOnOneToManyJoinedFieldWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with min on a one-to-many joined field, with filter.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + TotalPages: _min( + articles: { + field: pages, + filter: { + name: { + _eq: "To my dear readers" + } + } + } + ) + } + }`, + + ExpectedPatterns: minTypeIndexJoinPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "minNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "articles", + "childFieldName": "pages", + "filter": dataMap{ + "name": dataMap{ + "_eq": "To my dear readers", + }, + }, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "articles", + }, + }, + { + TargetNodeName: "scanNode", // inside of root + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "1", + "collectionName": "Article", + "filter": dataMap{ + "name": dataMap{ + "_eq": "To my dear readers", + }, + }, + "spans": []dataMap{ + { + "start": "/1", + "end": "/2", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestDefaultExplainRequest_WithMinOnOneToManyJoinedFieldWithManySources_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with min on a one-to-many joined field with many sources.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Author { + name + TotalPages: _min( + books: {field: pages}, + articles: {field: pages} + ) + } + }`, + + ExpectedPatterns: dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "parallelNode": []dataMap{ + { + "typeIndexJoin": normalTypeJoinPattern, + }, + { + "typeIndexJoin": normalTypeJoinPattern, + }, + }, + }, + }, + }, + }, + }, + }, + }, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "minNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "childFieldName": "pages", + "fieldName": "books", + "filter": nil, + }, + + { + "childFieldName": "pages", + "fieldName": "articles", + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + OccurancesToSkip: 0, + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "books", + }, + }, + { + TargetNodeName: "scanNode", // inside of 1st root type + OccurancesToSkip: 0, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of 1st subType (related type) + OccurancesToSkip: 1, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + { + TargetNodeName: "typeIndexJoin", + OccurancesToSkip: 1, + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "joinType": "typeJoinMany", + "rootName": immutable.Some("author"), + "subTypeName": "articles", + }, + }, + { + TargetNodeName: "scanNode", // inside of 2nd root type (AKA: subType's root) + OccurancesToSkip: 2, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "3", + "collectionName": "Author", + "filter": nil, + "spans": []dataMap{ + { + "start": "/3", + "end": "/4", + }, + }, + }, + }, + { + TargetNodeName: "scanNode", // inside of 2nd subType (AKA: subType's subtype) + OccurancesToSkip: 3, + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "1", + "collectionName": "Article", + "filter": nil, + "spans": []dataMap{ + { + "start": "/1", + "end": "/2", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/default/with_min_test.go b/tests/integration/explain/default/with_min_test.go new file mode 100644 index 0000000000..63da42909e --- /dev/null +++ b/tests/integration/explain/default/with_min_test.go @@ -0,0 +1,90 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_default + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +var minPattern = dataMap{ + "explain": dataMap{ + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "selectNode": dataMap{ + "scanNode": dataMap{}, + }, + }, + }, + }, + }, + }, +} + +func TestDefaultExplainRequest_WithMinOnInlineArrayField_ChildFieldWillBeEmpty(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (default) request with min on an inline array field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + testUtils.ExplainRequest{ + + Request: `query @explain { + Book { + name + MinChapterPages: _min(chapterPages: {}) + } + }`, + + ExpectedPatterns: minPattern, + + ExpectedTargets: []testUtils.PlanNodeTargetCase{ + { + TargetNodeName: "minNode", + IncludeChildNodes: false, + ExpectedAttributes: dataMap{ + "sources": []dataMap{ + { + "fieldName": "chapterPages", + "childFieldName": nil, + "filter": nil, + }, + }, + }, + }, + { + TargetNodeName: "scanNode", + IncludeChildNodes: true, // should be leaf of it's branch, so will have no child nodes. + ExpectedAttributes: dataMap{ + "collectionID": "2", + "collectionName": "Book", + "filter": nil, + "spans": []dataMap{ + { + "start": "/2", + "end": "/3", + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/execute/with_max_test.go b/tests/integration/explain/execute/with_max_test.go new file mode 100644 index 0000000000..139c86e210 --- /dev/null +++ b/tests/integration/explain/execute/with_max_test.go @@ -0,0 +1,138 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_execute + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +func TestExecuteExplainRequest_WithMaxOfInlineArrayField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (execute) request with max on an inline array.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + create2AddressDocuments(), + create2AuthorContactDocuments(), + create2AuthorDocuments(), + create3BookDocuments(), + + testUtils.ExplainRequest{ + Request: `query @explain(type: execute) { + Book { + name + MaxChapterPages: _max(chapterPages: {}) + } + }`, + + ExpectedFullGraph: dataMap{ + "explain": dataMap{ + "executionSuccess": true, + "sizeOfResult": 1, + "planExecutions": uint64(2), + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "iterations": uint64(4), + "selectNode": dataMap{ + "iterations": uint64(4), + "filterMatches": uint64(3), + "scanNode": dataMap{ + "iterations": uint64(4), + "docFetches": uint64(3), + "fieldFetches": uint64(5), + "indexFetches": uint64(0), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestExecuteExplainRequest_MaxOfRelatedOneToManyField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (execute) request with max of a related one to many field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + create2AddressDocuments(), + create2AuthorContactDocuments(), + create2AuthorDocuments(), + create3ArticleDocuments(), + + testUtils.ExplainRequest{ + Request: `query @explain(type: execute) { + Author { + name + MaxPages: _max( + articles: { + field: pages, + } + ) + } + }`, + + ExpectedFullGraph: dataMap{ + "explain": dataMap{ + "executionSuccess": true, + "sizeOfResult": 1, + "planExecutions": uint64(2), + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "maxNode": dataMap{ + "iterations": uint64(3), + "selectNode": dataMap{ + "iterations": uint64(3), + "filterMatches": uint64(2), + "typeIndexJoin": dataMap{ + "iterations": uint64(3), + "scanNode": dataMap{ + "iterations": uint64(3), + "docFetches": uint64(2), + "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(9), + "indexFetches": uint64(0), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/explain/execute/with_min_test.go b/tests/integration/explain/execute/with_min_test.go new file mode 100644 index 0000000000..204a055f32 --- /dev/null +++ b/tests/integration/explain/execute/with_min_test.go @@ -0,0 +1,138 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package test_explain_execute + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + explainUtils "github.com/sourcenetwork/defradb/tests/integration/explain" +) + +func TestExecuteExplainRequest_WithMinOfInlineArrayField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (execute) request with min on an inline array.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + + create2AddressDocuments(), + create2AuthorContactDocuments(), + create2AuthorDocuments(), + create3BookDocuments(), + + testUtils.ExplainRequest{ + Request: `query @explain(type: execute) { + Book { + name + MinChapterPages: _min(chapterPages: {}) + } + }`, + + ExpectedFullGraph: dataMap{ + "explain": dataMap{ + "executionSuccess": true, + "sizeOfResult": 1, + "planExecutions": uint64(2), + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "iterations": uint64(4), + "selectNode": dataMap{ + "iterations": uint64(4), + "filterMatches": uint64(3), + "scanNode": dataMap{ + "iterations": uint64(4), + "docFetches": uint64(3), + "fieldFetches": uint64(5), + "indexFetches": uint64(0), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} + +func TestExecuteExplainRequest_MinOfRelatedOneToManyField_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + + Description: "Explain (execute) request with min of a related one to many field.", + + Actions: []any{ + explainUtils.SchemaForExplainTests, + create2AddressDocuments(), + create2AuthorContactDocuments(), + create2AuthorDocuments(), + create3ArticleDocuments(), + + testUtils.ExplainRequest{ + Request: `query @explain(type: execute) { + Author { + name + MinPages: _min( + articles: { + field: pages, + } + ) + } + }`, + + ExpectedFullGraph: dataMap{ + "explain": dataMap{ + "executionSuccess": true, + "sizeOfResult": 1, + "planExecutions": uint64(2), + "operationNode": []dataMap{ + { + "selectTopNode": dataMap{ + "minNode": dataMap{ + "iterations": uint64(3), + "selectNode": dataMap{ + "iterations": uint64(3), + "filterMatches": uint64(2), + "typeIndexJoin": dataMap{ + "iterations": uint64(3), + "scanNode": dataMap{ + "iterations": uint64(3), + "docFetches": uint64(2), + "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(9), + "indexFetches": uint64(0), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + explainUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_max_doc_id_test.go b/tests/integration/query/inline_array/with_max_doc_id_test.go new file mode 100644 index 0000000000..3a473db9f2 --- /dev/null +++ b/tests/integration/query/inline_array/with_max_doc_id_test.go @@ -0,0 +1,51 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +// This test is meant to provide coverage of the planNode.Spans +// func by targeting a specific docID in the parent select. +func TestQueryInlineNillableFloatArray_WithDocIDAndMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with doc id, max of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users(docID: "bae-3f7e0f22-e253-53dd-b31b-df8b081292d9") { + name + _max(pageRatings: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": float64(10), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_max_filter_test.go b/tests/integration/query/inline_array/with_max_filter_test.go new file mode 100644 index 0000000000..8d3b1cee25 --- /dev/null +++ b/tests/integration/query/inline_array/with_max_filter_test.go @@ -0,0 +1,145 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, filtered max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, -1, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {filter: {_lt: 2}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with filter, max of nillable integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [-1, 2, null, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(testScores: {filter: {_lt: 2}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, filtered max of float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {filter: {_lt: 9}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with filter, max of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(pageRatings: {filter: {_lt: 9}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_max_limit_offset_order_test.go b/tests/integration/query/inline_array/with_max_limit_offset_order_test.go new file mode 100644 index 0000000000..7f18935d93 --- /dev/null +++ b/tests/integration/query/inline_array/with_max_limit_offset_order_test.go @@ -0,0 +1,281 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMaxWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0, 1, 2 + "_max": int64(2), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMaxWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 5, 2, 1 + "_max": int64(5), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMaxWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [2, null, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(testScores: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0, 1, 2 + "_max": int64(2), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMaxWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [null, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(testScores: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 5, 2, 1 + "_max": int64(5), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0.577, 2.718, 3.1425 + "_max": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 6.283, 3.1425, 2.718 + "_max": float64(6.283), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMaxWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, null, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(pageRatings: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0.577, 2.718, 3.1425 + "_max": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMaxWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, null, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(pageRatings: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 6.283, 3.1425, 2.718 + "_max": float64(6.283), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_max_limit_offset_test.go b/tests/integration/query/inline_array/with_max_limit_offset_test.go new file mode 100644 index 0000000000..83eb8ac8ff --- /dev/null +++ b/tests/integration/query/inline_array/with_max_limit_offset_test.go @@ -0,0 +1,49 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMaxWithOffsetWithLimit_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, offsetted limited max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {offset: 1, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": int64(5), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_max_test.go b/tests/integration/query/inline_array/with_max_test.go new file mode 100644 index 0000000000..77cf0c8b37 --- /dev/null +++ b/tests/integration/query/inline_array/with_max_test.go @@ -0,0 +1,305 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMaxAndNullArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of nil integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteIntegers": null + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMaxAndEmptyArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of empty integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteIntegers": [] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMaxAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, -1, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": int64(2), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMaxAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of nillable integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [-1, 2, null, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(testScores: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": int64(2), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxAndNullArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of nil float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": null + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxAndEmptyArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of empty float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": [] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMaxAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": [3.1425, 0.00000000001, 10] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_max": float64(10), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMaxAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _max(pageRatings: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": float64(10), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithDocIDMaxAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, max of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users(docID: "bae-3f7e0f22-e253-53dd-b31b-df8b081292d9") { + name + _max(pageRatings: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_max": float64(10), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_min_doc_id_test.go b/tests/integration/query/inline_array/with_min_doc_id_test.go new file mode 100644 index 0000000000..b8ad3d2c98 --- /dev/null +++ b/tests/integration/query/inline_array/with_min_doc_id_test.go @@ -0,0 +1,51 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +// This test is meant to provide coverage of the planNode.Spans +// func by targeting a specific docID in the parent select. +func TestQueryInlineNillableFloatArray_WithDocIDAndMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with doc id, min of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users(docID: "bae-3f7e0f22-e253-53dd-b31b-df8b081292d9") { + name + _min(pageRatings: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": float64(0.00000000001), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_min_filter_test.go b/tests/integration/query/inline_array/with_min_filter_test.go new file mode 100644 index 0000000000..50352dc83c --- /dev/null +++ b/tests/integration/query/inline_array/with_min_filter_test.go @@ -0,0 +1,145 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, filtered min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, -1, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {filter: {_gt: 0}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with filter, min of nillable integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [-1, 2, null, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(testScores: {filter: {_gt: 0}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, filtered min of float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {filter: {_gt: 1}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with filter, min of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(pageRatings: {filter: {_gt: 1}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": float64(3.1425), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_min_limit_offset_order_test.go b/tests/integration/query/inline_array/with_min_limit_offset_order_test.go new file mode 100644 index 0000000000..ed5adde4b8 --- /dev/null +++ b/tests/integration/query/inline_array/with_min_limit_offset_order_test.go @@ -0,0 +1,281 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMinWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0, 1, 2 + "_min": int64(0), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMinWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 5, 2, 1 + "_min": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMinWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [2, null, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(testScores: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0, 1, 2 + "_min": int64(0), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMinWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [null, 2, 5, 1, 0, 7] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(testScores: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 5, 2, 1 + "_min": int64(1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0.577, 2.718, 3.1425 + "_min": float64(0.577), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteFloats": [3.1425, 0.00000000001, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 6.283, 3.1425, 2.718 + "_min": float64(2.718), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMinWithOffsetWithLimitWithOrderAsc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, null, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(pageRatings: {offset: 1, limit: 3, order: ASC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 0.577, 2.718, 3.1425 + "_min": float64(0.577), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMinWithOffsetWithLimitWithOrderDesc_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, ordered offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, null, 10, 2.718, 0.577, 6.283] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(pageRatings: {offset: 1, limit: 3, order: DESC}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + // 6.283, 3.1425, 2.718 + "_min": float64(2.718), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_min_limit_offset_test.go b/tests/integration/query/inline_array/with_min_limit_offset_test.go new file mode 100644 index 0000000000..a8b55dc786 --- /dev/null +++ b/tests/integration/query/inline_array/with_min_limit_offset_test.go @@ -0,0 +1,49 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMinWithOffsetWithLimit_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array, offsetted limited min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, 5, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {offset: 1, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": int64(2), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/inline_array/with_min_test.go b/tests/integration/query/inline_array/with_min_test.go new file mode 100644 index 0000000000..8788181415 --- /dev/null +++ b/tests/integration/query/inline_array/with_min_test.go @@ -0,0 +1,273 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package inline_array + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryInlineIntegerArray_WithMinAndNullArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of nil integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteIntegers": null + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMinAndEmptyArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of empty integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteIntegers": [] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineIntegerArray_WithMinAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "favouriteIntegers": [-1, 2, -1, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteIntegers: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": int64(-1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableIntegerArray_WithMinAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of nillable integer array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "testScores": [-1, 2, null, 1, 0] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(testScores: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": int64(-1), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinAndNullArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of nil float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": null + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinAndEmptyArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of empty float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": [] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineFloatArray_WithMinAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "John", + "favouriteFloats": [3.1425, 0.00000000001, 10] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(favouriteFloats: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "John", + "_min": float64(0.00000000001), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQueryInlineNillableFloatArray_WithMinAndPopulatedArray_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple inline array with no filter, min of nillable float array", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "name": "Shahzad", + "pageRatings": [3.1425, 0.00000000001, 10, null] + }`, + }, + testUtils.Request{ + Request: `query { + Users { + name + _min(pageRatings: {}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "name": "Shahzad", + "_min": float64(0.00000000001), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/utils.go b/tests/integration/query/simple/utils.go index 0f2258e30e..b64686d600 100644 --- a/tests/integration/query/simple/utils.go +++ b/tests/integration/query/simple/utils.go @@ -31,7 +31,8 @@ func executeTestCase(t *testing.T, test testUtils.TestCase) { testUtils.ExecuteTestCase( t, testUtils.TestCase{ - Description: test.Description, + Description: test.Description, + SupportedMutationTypes: test.SupportedMutationTypes, Actions: append( []any{ testUtils.SchemaUpdate{ diff --git a/tests/integration/query/simple/with_group_average_max_test.go b/tests/integration/query/simple/with_group_average_max_test.go new file mode 100644 index 0000000000..847f0db941 --- /dev/null +++ b/tests/integration/query/simple/with_group_average_max_test.go @@ -0,0 +1,164 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuery_SimpleWithGroupByStringWithInnerGroupBooleanAndMaxOfAverageOfInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and max of average on int", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: _avg}) + _group (groupBy: [Verified]){ + Verified + _avg(_group: {field: Age}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": float64(34), + "_group": []map[string]any{ + { + "Verified": true, + "_avg": float64(28.5), + }, + { + "Verified": false, + "_avg": float64(34), + }, + }, + }, + { + "Name": "Carlo", + "_max": float64(55), + "_group": []map[string]any{ + { + "Verified": true, + "_avg": float64(55), + }, + }, + }, + { + "Name": "Alice", + "_max": float64(19), + "_group": []map[string]any{ + { + "Verified": false, + "_avg": float64(19), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerAverageAndMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, average and max on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _avg(_group: {field: Age}) + _max(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_avg": float64(35), + "_max": int64(38), + }, + { + "Name": "Alice", + "_avg": float64(-19), + "_max": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_average_min_test.go b/tests/integration/query/simple/with_group_average_min_test.go new file mode 100644 index 0000000000..c59c576b3b --- /dev/null +++ b/tests/integration/query/simple/with_group_average_min_test.go @@ -0,0 +1,164 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuery_SimpleWithGroupByStringWithInnerGroupBooleanAndMinOfAverageOfInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and min of average on int", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: _avg}) + _group (groupBy: [Verified]){ + Verified + _avg(_group: {field: Age}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": float64(28.5), + "_group": []map[string]any{ + { + "Verified": true, + "_avg": float64(28.5), + }, + { + "Verified": false, + "_avg": float64(34), + }, + }, + }, + { + "Name": "Carlo", + "_min": float64(55), + "_group": []map[string]any{ + { + "Verified": true, + "_avg": float64(55), + }, + }, + }, + { + "Name": "Alice", + "_min": float64(19), + "_group": []map[string]any{ + { + "Verified": false, + "_avg": float64(19), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerAverageAndMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, average and min on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _avg(_group: {field: Age}) + _min(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_avg": float64(35), + "_min": int64(32), + }, + { + "Name": "Alice", + "_avg": float64(-19), + "_min": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_count_max_test.go b/tests/integration/query/simple/with_group_count_max_test.go new file mode 100644 index 0000000000..82749a3b76 --- /dev/null +++ b/tests/integration/query/simple/with_group_count_max_test.go @@ -0,0 +1,112 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMaxOfCount_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and max of count", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: _count}) + _group (groupBy: [Verified]){ + Verified + _count(_group: {}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(2), + "_group": []map[string]any{ + { + "Verified": true, + "_count": int(2), + }, + { + "Verified": false, + "_count": int(1), + }, + }, + }, + { + "Name": "Carlo", + "_max": int64(1), + "_group": []map[string]any{ + { + "Verified": true, + "_count": int(1), + }, + }, + }, + { + "Name": "Alice", + "_max": int64(1), + "_group": []map[string]any{ + { + "Verified": false, + "_count": int(1), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_count_min_test.go b/tests/integration/query/simple/with_group_count_min_test.go new file mode 100644 index 0000000000..4890ed527c --- /dev/null +++ b/tests/integration/query/simple/with_group_count_min_test.go @@ -0,0 +1,112 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMinOfCount_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and min of count", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: _count}) + _group (groupBy: [Verified]){ + Verified + _count(_group: {}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(1), + "_group": []map[string]any{ + { + "Verified": true, + "_count": int(2), + }, + { + "Verified": false, + "_count": int(1), + }, + }, + }, + { + "Name": "Carlo", + "_min": int64(1), + "_group": []map[string]any{ + { + "Verified": true, + "_count": int(1), + }, + }, + }, + { + "Name": "Alice", + "_min": int64(1), + "_group": []map[string]any{ + { + "Verified": false, + "_count": int(1), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_max_filter_test.go b/tests/integration/query/simple/with_group_max_filter_test.go new file mode 100644 index 0000000000..13db6c6e10 --- /dev/null +++ b/tests/integration/query/simple/with_group_max_filter_test.go @@ -0,0 +1,294 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByNumberWithoutRenderedGroupAndChildMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on non-rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _max(_group: {field: Age, filter: {Age: {_gt: 26}}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_max": int64(32), + }, + { + "Age": int64(19), + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupAndChildMaxWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _max(_group: {field: Age, filter: {Age: {_gt: 26}}}) + _group { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_max": int64(32), + "_group": []map[string]any{ + { + "Name": "Bob", + }, + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_max": nil, + "_group": []map[string]any{ + { + "Name": "Alice", + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupWithFilterAndChildMaxWithMatchingFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on rendered, matching filtered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _max(_group: {field: Age, filter: {Name: {_eq: "John"}}}) + _group(filter: {Name: {_eq: "John"}}) { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_max": int64(32), + "_group": []map[string]any{ + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_max": nil, + "_group": []map[string]any{}, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupWithFilterAndChildMaxWithDifferentFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on non-rendered, different filtered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _max(_group: {field: Age, filter: {Age: {_gt: 26}}}) + _group(filter: {Name: {_eq: "John"}}) { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_max": int64(32), + "_group": []map[string]any{ + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_max": nil, + "_group": []map[string]any{}, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithoutRenderedGroupAndChildMaxWithDifferentFilters_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on non-rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + S1: _max(_group: {field: Age, filter: {Age: {_gt: 26}}}) + S2: _max(_group: {field: Age, filter: {Age: {_lt: 26}}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "S1": int64(32), + "S2": nil, + }, + { + "Age": int64(19), + "S1": nil, + "S2": int64(19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_max_limit_offset_test.go b/tests/integration/query/simple/with_group_max_limit_offset_test.go new file mode 100644 index 0000000000..601aa3a6f7 --- /dev/null +++ b/tests/integration/query/simple/with_group_max_limit_offset_test.go @@ -0,0 +1,72 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMaxWithLimitAndOffset_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, offsetted limited max on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 28 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: Age, offset: 1, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(38), + }, + { + "Name": "Alice", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_max_limit_test.go b/tests/integration/query/simple/with_group_max_limit_test.go new file mode 100644 index 0000000000..924032ba43 --- /dev/null +++ b/tests/integration/query/simple/with_group_max_limit_test.go @@ -0,0 +1,72 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMaxWithLimit_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, limited max on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 28 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: Age, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(38), + }, + { + "Name": "Alice", + "_max": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_max_test.go b/tests/integration/query/simple/with_group_max_test.go new file mode 100644 index 0000000000..f501223312 --- /dev/null +++ b/tests/integration/query/simple/with_group_max_test.go @@ -0,0 +1,577 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndMaxOfUndefined_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with max on unspecified field", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.Request{ + Request: `query { + Users (groupBy: [Name]) { + Name + _max + } + }`, + ExpectedError: "aggregate must be provided with a property to aggregate", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMaxOnEmptyCollection_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, max on non-rendered group, empty collection", + Actions: []any{ + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _max(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{}, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, max on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(38), + }, + { + "Name": "Alice", + "_max": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildNilMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, max on non-rendered group nil and integer values", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + // Age is undefined here + Doc: `{ + "Name": "John" + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(32), + }, + { + "Name": "Alice", + "_max": int64(19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMaxOfMaxOfInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and max of max on int", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: _max}) + _group (groupBy: [Verified]){ + Verified + _max(_group: {field: Age}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": int64(34), + "_group": []map[string]any{ + { + "Verified": true, + "_max": int64(32), + }, + { + "Verified": false, + "_max": int64(34), + }, + }, + }, + { + "Name": "Carlo", + "_max": int64(55), + "_group": []map[string]any{ + { + "Verified": true, + "_max": int64(55), + }, + }, + }, + { + "Name": "Alice", + "_max": int64(19), + "_group": []map[string]any{ + { + "Verified": false, + "_max": int64(19), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildEmptyFloatMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, max on non-rendered group float (default) value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.89 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice" + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: HeightM}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": float64(1.89), + }, + { + "Name": "Alice", + "_max": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildFloatMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, max on non-rendered group float value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.89 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: HeightM}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_max": float64(1.89), + }, + { + "Name": "Alice", + "_max": float64(2.04), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMaxOfMaxOfFloat_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and max of max on float", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.61, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 2.22, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "HeightM": 1.74, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: _max}) + _group (groupBy: [Verified]){ + Verified + _max(_group: {field: HeightM}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "Alice", + "_max": float64(2.04), + "_group": []map[string]any{ + { + "Verified": false, + "_max": float64(2.04), + }, + }, + }, + { + "Name": "John", + "_max": float64(2.22), + "_group": []map[string]any{ + { + "Verified": true, + "_max": float64(1.82), + }, + { + "Verified": false, + "_max": float64(2.22), + }, + }, + }, + { + "Name": "Carlo", + "_max": float64(1.74), + "_group": []map[string]any{ + { + "Verified": true, + "_max": float64(1.74), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMaxOfMaxOfMaxOfFloat_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and max of max of max of float", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82, + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.61, + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 2.22, + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "HeightM": 1.74, + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04, + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _max(_group: {field: _max}) + _group (groupBy: [Verified]){ + Verified + _max(_group: {field: HeightM}) + _group (groupBy: [Age]){ + Age + _max(_group: {field: HeightM}) + } + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "Carlo", + "_max": float64(1.74), + "_group": []map[string]any{ + { + "Verified": true, + "_max": float64(1.74), + "_group": []map[string]any{ + { + "Age": int64(55), + "_max": float64(1.74), + }, + }, + }, + }, + }, + { + "Name": "Alice", + "_max": float64(2.04), + "_group": []map[string]any{ + { + "Verified": false, + "_max": float64(2.04), + "_group": []map[string]any{ + { + "Age": int64(19), + "_max": float64(2.04), + }, + }, + }, + }, + }, + { + "Name": "John", + "_max": float64(2.22), + "_group": []map[string]any{ + { + "Verified": true, + "_max": float64(1.82), + "_group": []map[string]any{ + { + "Age": int64(32), + "_max": float64(1.61), + }, + { + "Age": int64(25), + "_max": float64(1.82), + }, + }, + }, + { + "Verified": false, + "_max": float64(2.22), + "_group": []map[string]any{ + { + "Age": int64(34), + "_max": float64(2.22), + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_min_filter_test.go b/tests/integration/query/simple/with_group_min_filter_test.go new file mode 100644 index 0000000000..21332bc827 --- /dev/null +++ b/tests/integration/query/simple/with_group_min_filter_test.go @@ -0,0 +1,294 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByNumberWithoutRenderedGroupAndChildMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on non-rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _min(_group: {field: Age, filter: {Age: {_gt: 26}}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_min": int64(32), + }, + { + "Age": int64(19), + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupAndChildMinWithFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _min(_group: {field: Age, filter: {Age: {_gt: 26}}}) + _group { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_min": int64(32), + "_group": []map[string]any{ + { + "Name": "Bob", + }, + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_min": nil, + "_group": []map[string]any{ + { + "Name": "Alice", + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupWithFilterAndChildMinWithMatchingFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on rendered, matching filtered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _min(_group: {field: Age, filter: {Name: {_eq: "John"}}}) + _group(filter: {Name: {_eq: "John"}}) { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_min": int64(32), + "_group": []map[string]any{ + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_min": nil, + "_group": []map[string]any{}, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithRenderedGroupWithFilterAndChildMinWithDifferentFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on non-rendered, different filtered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _min(_group: {field: Age, filter: {Age: {_gt: 26}}}) + _group(filter: {Name: {_eq: "John"}}) { + Name + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "_min": int64(32), + "_group": []map[string]any{ + { + "Name": "John", + }, + }, + }, + { + "Age": int64(19), + "_min": nil, + "_group": []map[string]any{}, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByNumberWithoutRenderedGroupAndChildMinWithDifferentFilters_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on non-rendered, unfiltered group", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + S1: _min(_group: {field: Age, filter: {Age: {_gt: 26}}}) + S2: _min(_group: {field: Age, filter: {Age: {_lt: 26}}}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Age": int64(32), + "S1": int64(32), + "S2": nil, + }, + { + "Age": int64(19), + "S1": nil, + "S2": int64(19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_min_limit_offset_test.go b/tests/integration/query/simple/with_group_min_limit_offset_test.go new file mode 100644 index 0000000000..b7ac741500 --- /dev/null +++ b/tests/integration/query/simple/with_group_min_limit_offset_test.go @@ -0,0 +1,72 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMinWithLimitAndOffset_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, offsetted limited min on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 28 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: Age, offset: 1, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(32), + }, + { + "Name": "Alice", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_min_limit_test.go b/tests/integration/query/simple/with_group_min_limit_test.go new file mode 100644 index 0000000000..c2f350d218 --- /dev/null +++ b/tests/integration/query/simple/with_group_min_limit_test.go @@ -0,0 +1,72 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMinWithLimit_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, limited min on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 28 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: Age, limit: 2}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(28), + }, + { + "Name": "Alice", + "_min": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_group_min_test.go b/tests/integration/query/simple/with_group_min_test.go new file mode 100644 index 0000000000..ac8d2e3ed9 --- /dev/null +++ b/tests/integration/query/simple/with_group_min_test.go @@ -0,0 +1,577 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndMinOfUndefined_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with min on unspecified field", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.Request{ + Request: `query { + Users (groupBy: [Name]) { + Name + _min + } + }`, + ExpectedError: "aggregate must be provided with a property to aggregate", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMinOnEmptyCollection_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by number, no children, min on non-rendered group, empty collection", + Actions: []any{ + testUtils.Request{ + Request: `query { + Users(groupBy: [Age]) { + Age + _min(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{}, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildIntegerMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, min on non-rendered group integer value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 38 + }`, + }, + testUtils.CreateDoc{ + // It is important to test negative values here, due to the auto-typing of numbers + Doc: `{ + "Name": "Alice", + "Age": -19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(32), + }, + { + "Name": "Alice", + "_min": int64(-19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildNilMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, min on non-rendered group nil and integer values", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32 + }`, + }, + testUtils.CreateDoc{ + // Age is undefined here + Doc: `{ + "Name": "John" + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: Age}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(32), + }, + { + "Name": "Alice", + "_min": int64(19), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMinOfMinOfInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and min of min on int", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: _min}) + _group (groupBy: [Verified]){ + Verified + _min(_group: {field: Age}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": int64(25), + "_group": []map[string]any{ + { + "Verified": true, + "_min": int64(25), + }, + { + "Verified": false, + "_min": int64(34), + }, + }, + }, + { + "Name": "Carlo", + "_min": int64(55), + "_group": []map[string]any{ + { + "Verified": true, + "_min": int64(55), + }, + }, + }, + { + "Name": "Alice", + "_min": int64(19), + "_group": []map[string]any{ + { + "Verified": false, + "_min": int64(19), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildEmptyFloatMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, min on non-rendered group float (default) value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.89 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice" + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: HeightM}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": float64(1.82), + }, + { + "Name": "Alice", + "_min": nil, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithoutRenderedGroupAndChildFloatMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, min on non-rendered group float value", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.89 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04 + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: HeightM}) + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "John", + "_min": float64(1.82), + }, + { + "Name": "Alice", + "_min": float64(2.04), + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMinOfMinOfFloat_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and min of min on float", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.61, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 2.22, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "HeightM": 1.74, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: _min}) + _group (groupBy: [Verified]){ + Verified + _min(_group: {field: HeightM}) + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "Alice", + "_min": float64(2.04), + "_group": []map[string]any{ + { + "Verified": false, + "_min": float64(2.04), + }, + }, + }, + { + "Name": "John", + "_min": float64(1.61), + "_group": []map[string]any{ + { + "Verified": true, + "_min": float64(1.61), + }, + { + "Verified": false, + "_min": float64(2.22), + }, + }, + }, + { + "Name": "Carlo", + "_min": float64(1.74), + "_group": []map[string]any{ + { + "Verified": true, + "_min": float64(1.74), + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithGroupByStringWithInnerGroupBooleanAndMinOfMinOfMinOfFloat_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with group by string, with child group by boolean, and min of min of min of float", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.82, + "Age": 25, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 1.61, + "Age": 32, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "HeightM": 2.22, + "Age": 34, + "Verified": false + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Carlo", + "HeightM": 1.74, + "Age": 55, + "Verified": true + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "HeightM": 2.04, + "Age": 19, + "Verified": false + }`, + }, + testUtils.Request{ + Request: `query { + Users(groupBy: [Name]) { + Name + _min(_group: {field: _min}) + _group (groupBy: [Verified]){ + Verified + _min(_group: {field: HeightM}) + _group (groupBy: [Age]){ + Age + _min(_group: {field: HeightM}) + } + } + } + }`, + Results: map[string]any{ + "Users": []map[string]any{ + { + "Name": "Carlo", + "_min": float64(1.74), + "_group": []map[string]any{ + { + "Verified": true, + "_min": float64(1.74), + "_group": []map[string]any{ + { + "Age": int64(55), + "_min": float64(1.74), + }, + }, + }, + }, + }, + { + "Name": "Alice", + "_min": float64(2.04), + "_group": []map[string]any{ + { + "Verified": false, + "_min": float64(2.04), + "_group": []map[string]any{ + { + "Age": int64(19), + "_min": float64(2.04), + }, + }, + }, + }, + }, + { + "Name": "John", + "_min": float64(1.61), + "_group": []map[string]any{ + { + "Verified": true, + "_min": float64(1.61), + "_group": []map[string]any{ + { + "Age": int64(32), + "_min": float64(1.61), + }, + { + "Age": int64(25), + "_min": float64(1.82), + }, + }, + }, + { + "Verified": false, + "_min": float64(2.22), + "_group": []map[string]any{ + { + "Age": int64(34), + "_min": float64(2.22), + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_max_filter_test.go b/tests/integration/query/simple/with_max_filter_test.go new file mode 100644 index 0000000000..78f275ca46 --- /dev/null +++ b/tests/integration/query/simple/with_max_filter_test.go @@ -0,0 +1,53 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithMaxFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with max filter", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 21 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 30 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 32 + }`, + }, + testUtils.Request{ + Request: `query { + _max(Users: {field: Age, filter: {Age: {_lt: 32}}}) + }`, + Results: map[string]any{ + "_max": int64(30), + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_max_test.go b/tests/integration/query/simple/with_max_test.go new file mode 100644 index 0000000000..bdb47b6f8c --- /dev/null +++ b/tests/integration/query/simple/with_max_test.go @@ -0,0 +1,129 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "math" + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + + "github.com/sourcenetwork/immutable" +) + +func TestQuerySimple_WithMaxOnUndefinedObject_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query max on undefined object", + Actions: []any{ + testUtils.Request{ + Request: `query { + _max + }`, + ExpectedError: "aggregate must be provided with a property to aggregate", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMaxOnUndefinedField_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query max on undefined field", + Actions: []any{ + testUtils.Request{ + Request: `query { + _max(Users: {}) + }`, + ExpectedError: "Argument \"Users\" has invalid value {}.\nIn field \"field\": Expected \"UsersNumericFieldsArg!\", found null.", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMaxOnEmptyCollection_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query max on empty", + Actions: []any{ + testUtils.Request{ + Request: `query { + _max(Users: {field: Age}) + }`, + Results: map[string]any{ + "_max": nil, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMax_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query max", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 21 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 30 + }`, + }, + testUtils.Request{ + Request: `query { + _max(Users: {field: Age}) + }`, + Results: map[string]any{ + "_max": int64(30), + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMaxAndMaxValueInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + SupportedMutationTypes: immutable.Some([]testUtils.MutationType{ + // GraphQL does not support 64 bit int + testUtils.CollectionSaveMutationType, + testUtils.CollectionNamedMutationType, + }), + Description: "Simple query max and max value int", + Actions: []any{ + testUtils.CreateDoc{ + DocMap: map[string]any{ + "Name": "John", + "Age": int64(math.MaxInt64), + }, + }, + testUtils.Request{ + Request: `query { + _max(Users: {field: Age}) + }`, + Results: map[string]any{ + "_max": int64(math.MaxInt64), + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_min_filter_test.go b/tests/integration/query/simple/with_min_filter_test.go new file mode 100644 index 0000000000..cd429e3361 --- /dev/null +++ b/tests/integration/query/simple/with_min_filter_test.go @@ -0,0 +1,53 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQuerySimple_WithMinFilter_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query with min filter", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 21 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 30 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Alice", + "Age": 32 + }`, + }, + testUtils.Request{ + Request: `query { + _min(Users: {field: Age, filter: {Age: {_gt: 21}}}) + }`, + Results: map[string]any{ + "_min": int64(30), + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/query/simple/with_min_test.go b/tests/integration/query/simple/with_min_test.go new file mode 100644 index 0000000000..feb8e54e2f --- /dev/null +++ b/tests/integration/query/simple/with_min_test.go @@ -0,0 +1,129 @@ +// Copyright 2024 Democratized Data Foundation +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package simple + +import ( + "math" + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" + + "github.com/sourcenetwork/immutable" +) + +func TestQuerySimple_WithMinOnUndefinedObject_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query min on undefined object", + Actions: []any{ + testUtils.Request{ + Request: `query { + _min + }`, + ExpectedError: "aggregate must be provided with a property to aggregate", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMinOnUndefinedField_ReturnsError(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query min on undefined field", + Actions: []any{ + testUtils.Request{ + Request: `query { + _min(Users: {}) + }`, + ExpectedError: "Argument \"Users\" has invalid value {}.\nIn field \"field\": Expected \"UsersNumericFieldsArg!\", found null.", + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMinOnEmptyCollection_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query min on empty", + Actions: []any{ + testUtils.Request{ + Request: `query { + _min(Users: {field: Age}) + }`, + Results: map[string]any{ + "_min": nil, + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMin_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + Description: "Simple query min", + Actions: []any{ + testUtils.CreateDoc{ + Doc: `{ + "Name": "John", + "Age": 21 + }`, + }, + testUtils.CreateDoc{ + Doc: `{ + "Name": "Bob", + "Age": 30 + }`, + }, + testUtils.Request{ + Request: `query { + _min(Users: {field: Age}) + }`, + Results: map[string]any{ + "_min": int64(21), + }, + }, + }, + } + + executeTestCase(t, test) +} + +func TestQuerySimple_WithMinAndMaxValueInt_Succeeds(t *testing.T) { + test := testUtils.TestCase{ + SupportedMutationTypes: immutable.Some([]testUtils.MutationType{ + // GraphQL does not support 64 bit int + testUtils.CollectionSaveMutationType, + testUtils.CollectionNamedMutationType, + }), + Description: "Simple query min and max value int", + Actions: []any{ + testUtils.CreateDoc{ + DocMap: map[string]any{ + "Name": "John", + "Age": int64(math.MaxInt64), + }, + }, + testUtils.Request{ + Request: `query { + _max(Users: {field: Age}) + }`, + Results: map[string]any{ + "_max": int64(math.MaxInt64), + }, + }, + }, + } + + executeTestCase(t, test) +} diff --git a/tests/integration/schema/default_fields.go b/tests/integration/schema/default_fields.go index 6462ef6066..18c09975a2 100644 --- a/tests/integration/schema/default_fields.go +++ b/tests/integration/schema/default_fields.go @@ -119,6 +119,20 @@ var aggregateFields = fields{ "name": "Float", }, }, + map[string]any{ + "name": "_max", + "type": map[string]any{ + "kind": "SCALAR", + "name": "Float", + }, + }, + map[string]any{ + "name": "_min", + "type": map[string]any{ + "kind": "SCALAR", + "name": "Float", + }, + }, map[string]any{ "name": "_count", "type": map[string]any{