diff --git a/docs/data_format_changes/i3417-sec-index-compound-filter-conditions.md b/docs/data_format_changes/i3417-sec-index-compound-filter-conditions.md new file mode 100644 index 0000000000..b236fa96b0 --- /dev/null +++ b/docs/data_format_changes/i3417-sec-index-compound-filter-conditions.md @@ -0,0 +1,3 @@ +# Enabled compound filter conditions for secondary indexes + +Changed the way fieldFetched are counted for explain directive diff --git a/internal/connor/all.go b/internal/connor/all.go index bf607b583b..12c2209bad 100644 --- a/internal/connor/all.go +++ b/internal/connor/all.go @@ -43,7 +43,7 @@ func all(condition, data any) (bool, error) { func allSlice[T any](condition any, data []T) (bool, error) { for _, c := range data { - m, err := eq(condition, c) + m, err := eq(condition, c, true) if err != nil { return false, err } else if !m { diff --git a/internal/connor/and.go b/internal/connor/and.go index d302617dc8..2cef6e23ce 100644 --- a/internal/connor/and.go +++ b/internal/connor/and.go @@ -8,7 +8,7 @@ func and(condition, data any) (bool, error) { switch cn := condition.(type) { case []any: for _, c := range cn { - if m, err := eq(c, data); err != nil { + if m, err := eq(c, data, true); err != nil { return false, err } else if !m { return false, nil diff --git a/internal/connor/any.go b/internal/connor/any.go index ecd16ce992..63f20499a3 100644 --- a/internal/connor/any.go +++ b/internal/connor/any.go @@ -43,7 +43,7 @@ func anyOp(condition, data any) (bool, error) { func anySlice[T any](condition any, data []T) (bool, error) { for _, c := range data { - m, err := eq(condition, c) + m, err := eq(condition, c, true) if err != nil { return false, err } else if m { diff --git a/internal/connor/connor.go b/internal/connor/connor.go index cdb49d2973..6260b32aa7 100644 --- a/internal/connor/connor.go +++ b/internal/connor/connor.go @@ -49,13 +49,17 @@ func IsOpSimple(op string) bool { // Match is the default method used in Connor to match some data to a // set of conditions. func Match(conditions map[FilterKey]any, data any) (bool, error) { - return eq(conditions, data) + return eq(conditions, data, true) } // matchWith can be used to specify the exact operator to use when performing // a match operation. This is primarily used when building custom operators or // if you wish to override the behavior of another operator. -func matchWith(op string, conditions, data any) (bool, error) { +// It also takes a propExists boolean to indicate if the property exists in the data. +// It's needed because the behavior of the operators can change if the property doesn't exist. +// For example, _ne operator should return true if the property doesn't exist. +// This can also be used in the future if we introduce operators line _has. +func matchWith(op string, conditions, data any, propExists bool) (bool, error) { switch op { case AndOp: return and(conditions, data) @@ -64,7 +68,7 @@ func matchWith(op string, conditions, data any) (bool, error) { case AllOp: return all(conditions, data) case EqualOp, AliasOp: - return eq(conditions, data) + return eq(conditions, data, propExists) case GreaterOrEqualOp: return ge(conditions, data) case GreaterOp: @@ -76,7 +80,7 @@ func matchWith(op string, conditions, data any) (bool, error) { case LesserOp: return lt(conditions, data) case NotEqualOp: - return ne(conditions, data) + return ne(conditions, data, propExists) case NotInOp: return nin(conditions, data) case OrOp: diff --git a/internal/connor/eq.go b/internal/connor/eq.go index 1caa43e81b..d6944573dd 100644 --- a/internal/connor/eq.go +++ b/internal/connor/eq.go @@ -11,9 +11,12 @@ import ( "github.com/sourcenetwork/defradb/internal/core" ) -// eq is an operator which performs object equality -// tests. -func eq(condition, data any) (bool, error) { +// eq is an operator which performs object equality tests. +// It also takes a propExists boolean to indicate if the property exists in the data. +// It's needed because the behavior of the operators can change if the property doesn't exist. +// For example, _ne operator should return true if the property doesn't exist. +// This can also be used in the future if we introduce operators line _has. +func eq(condition, data any, propExists bool) (bool, error) { switch arr := data.(type) { case []core.Doc: return anySlice(condition, arr) @@ -34,11 +37,12 @@ func eq(condition, data any) (bool, error) { switch cn := condition.(type) { case map[FilterKey]any: for prop, cond := range cn { - d, op, err := prop.PropertyAndOperator(data, EqualOp) + res, err := prop.PropertyAndOperator(data, EqualOp) if err != nil { return false, err } - m, err := matchWith(op, cond, d) + // If the property doesn't exist, we should pass it forward to nested operators. + m, err := matchWith(res.Operator, cond, res.Data, !res.MissProp && propExists) if err != nil { return false, err } @@ -85,7 +89,7 @@ func objectsEqual(condition map[string]any, data any) (bool, error) { return false, nil } for k, v := range d { - m, err := eq(condition[k], v) + m, err := eq(condition[k], v, true) if err != nil { return false, err } else if !m { diff --git a/internal/connor/in.go b/internal/connor/in.go index b9978b4bae..8a70826c74 100644 --- a/internal/connor/in.go +++ b/internal/connor/in.go @@ -8,7 +8,7 @@ func in(conditions, data any) (bool, error) { switch cn := conditions.(type) { case []any: for _, ce := range cn { - if m, err := eq(ce, data); err != nil { + if m, err := eq(ce, data, true); err != nil { return false, err } else if m { return true, nil diff --git a/internal/connor/key.go b/internal/connor/key.go index 98e8d747ea..45c9dcc291 100644 --- a/internal/connor/key.go +++ b/internal/connor/key.go @@ -1,13 +1,23 @@ package connor +// KeyResult represents the result of a filter key operation. +type KeyResult struct { + // Data is the data that should be used to filter the value matching the key. + Data any + // MissProp is true if the key is missing a property, otherwise false. + // It's relevant for object of dynamic type, like JSON. + MissProp bool + // Operator is the operator that should be used to filter the value matching the key. + // If the key does not have an operator the given defaultOp will be returned. + Operator string +} + // FilterKey represents a type that may be used as a map key // in a filter. type FilterKey interface { - // PropertyAndOperator returns the data and operator that should be used - // to filter the value matching this key. - // - // If the key does not have an operator the given defaultOp will be returned. - PropertyAndOperator(data any, defaultOp string) (any, string, error) + // PropertyAndOperator returns [KeyResult] that contains data and operator that should be + // used to filter the value matching this key. + PropertyAndOperator(data any, defaultOp string) (KeyResult, error) // Equal returns true if other is equal, otherwise returns false. Equal(other FilterKey) bool } diff --git a/internal/connor/ne.go b/internal/connor/ne.go index a677229d93..8751e00fd5 100644 --- a/internal/connor/ne.go +++ b/internal/connor/ne.go @@ -2,8 +2,13 @@ package connor // ne performs object inequality comparisons by inverting // the result of the EqualOperator for non-error cases. -func ne(conditions, data any) (bool, error) { - m, err := eq(conditions, data) +func ne(conditions, data any, propExists bool) (bool, error) { + // _ne operator should return false if the property does not exist. + if !propExists { + return false, nil + } + + m, err := eq(conditions, data, propExists) if err != nil { return false, err diff --git a/internal/connor/none.go b/internal/connor/none.go index 742d4dc977..b135b0da85 100644 --- a/internal/connor/none.go +++ b/internal/connor/none.go @@ -41,7 +41,7 @@ func none(condition, data any) (bool, error) { func noneSlice[T any](condition any, data []T) (bool, error) { for _, c := range data { - m, err := eq(condition, c) + m, err := eq(condition, c, true) if err != nil { return false, err } else if m { diff --git a/internal/connor/not.go b/internal/connor/not.go index 96fcd87ff8..49309ca415 100644 --- a/internal/connor/not.go +++ b/internal/connor/not.go @@ -3,7 +3,7 @@ package connor // not is an operator which performs object equality test // and returns the inverse of the result. func not(condition, data any) (bool, error) { - m, err := eq(condition, data) + m, err := eq(condition, data, true) if err != nil { return false, err } diff --git a/internal/connor/not_test.go b/internal/connor/not_test.go index 959ef04177..c5fabfcfe5 100644 --- a/internal/connor/not_test.go +++ b/internal/connor/not_test.go @@ -34,8 +34,8 @@ type operator struct { Operation string } -func (k *operator) PropertyAndOperator(data any, defaultOp string) (any, string, error) { - return data, k.Operation, nil +func (k *operator) PropertyAndOperator(data any, defaultOp string) (KeyResult, error) { + return KeyResult{Data: data, Operator: k.Operation}, nil } func (k *operator) Equal(other FilterKey) bool { diff --git a/internal/connor/or.go b/internal/connor/or.go index c15e27393f..c8555fdf48 100644 --- a/internal/connor/or.go +++ b/internal/connor/or.go @@ -8,7 +8,7 @@ func or(condition, data any) (bool, error) { switch cn := condition.(type) { case []any: for _, c := range cn { - if m, err := eq(c, data); err != nil { + if m, err := eq(c, data, true); err != nil { return false, err } else if m { return true, nil diff --git a/internal/db/collection_get.go b/internal/db/collection_get.go index c67f30e9b5..94806d47de 100644 --- a/internal/db/collection_get.go +++ b/internal/db/collection_get.go @@ -13,6 +13,8 @@ package db import ( "context" + "github.com/sourcenetwork/immutable" + "github.com/sourcenetwork/defradb/acp/identity" "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/internal/db/base" @@ -63,7 +65,8 @@ func (c *collection) get( // create a new document fetcher df := c.newFetcher() // initialize it with the primary index - err := df.Init(ctx, identity.FromContext(ctx), txn, c.db.acp, c, fields, nil, nil, showDeleted) + err := df.Init(ctx, identity.FromContext(ctx), txn, c.db.acp, immutable.Option[client.IndexDescription]{}, + c, fields, nil, nil, showDeleted) if err != nil { _ = df.Close() return nil, err diff --git a/internal/db/collection_index.go b/internal/db/collection_index.go index 7347ff3348..227942394c 100644 --- a/internal/db/collection_index.go +++ b/internal/db/collection_index.go @@ -311,6 +311,7 @@ func (c *collection) iterateAllDocs( identity.FromContext(ctx), txn, c.db.acp, + immutable.None[client.IndexDescription](), c, fields, nil, diff --git a/internal/db/fetcher/document.go b/internal/db/fetcher/document.go index f3b638e26e..7ec9a66f45 100644 --- a/internal/db/fetcher/document.go +++ b/internal/db/fetcher/document.go @@ -135,6 +135,9 @@ func (f *documentFetcher) GetFields() (immutable.Option[EncodedDocument], error) for { res, ok := f.kvResultsIter.NextSync() + if res.Error != nil { + return immutable.None[EncodedDocument](), res.Error + } if !ok { break } @@ -179,13 +182,16 @@ func (f *documentFetcher) appendKV(doc *encodedDocument, kv keyValue) error { return err } + // we count the fields fetched here instead of after checking if the field was requested + // because we need to count all fields fetched to see more accurate picture of the performance + // of the query + f.execInfo.FieldsFetched++ + fieldDesc, ok := f.fieldsByID[fieldID] if !ok { return nil } - f.execInfo.FieldsFetched++ - doc.properties[fieldDesc] = &encProperty{ Desc: fieldDesc, Raw: kv.Value, diff --git a/internal/db/fetcher/fetcher.go b/internal/db/fetcher/fetcher.go index 61c4ef5166..0b58a39d6c 100644 --- a/internal/db/fetcher/fetcher.go +++ b/internal/db/fetcher/fetcher.go @@ -56,6 +56,7 @@ type Fetcher interface { identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, acp immutable.Option[acp.ACP], + index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, diff --git a/internal/db/fetcher/indexer.go b/internal/db/fetcher/indexer.go index ec3c5dd4e9..414ebfbce9 100644 --- a/internal/db/fetcher/indexer.go +++ b/internal/db/fetcher/indexer.go @@ -12,68 +12,68 @@ package fetcher import ( "context" - "errors" "github.com/sourcenetwork/immutable" - "github.com/sourcenetwork/defradb/acp" - acpIdentity "github.com/sourcenetwork/defradb/acp/identity" "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/datastore" + "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/internal/core" "github.com/sourcenetwork/defradb/internal/db/base" "github.com/sourcenetwork/defradb/internal/keys" + "github.com/sourcenetwork/defradb/internal/planner/filter" "github.com/sourcenetwork/defradb/internal/planner/mapper" ) -// IndexFetcher is a fetcher that fetches documents by index. +// indexFetcher is a fetcher that fetches documents by index. // It fetches only the indexed field and the rest of the fields are fetched by the internal fetcher. -type IndexFetcher struct { - docFetcher Fetcher - col client.Collection +type indexFetcher struct { + ctx context.Context txn datastore.Txn + col client.Collection indexFilter *mapper.Filter - doc *encodedDocument mapping *core.DocumentMapping indexedFields []client.FieldDefinition - docFields []client.FieldDefinition + fieldsByID map[uint32]client.FieldDefinition indexDesc client.IndexDescription indexIter indexIterator - execInfo ExecInfo + currentDocID immutable.Option[string] + execInfo *ExecInfo } -var _ Fetcher = (*IndexFetcher)(nil) +var _ fetcher = (*indexFetcher)(nil) -// NewIndexFetcher creates a new IndexFetcher. -func NewIndexFetcher( - docFetcher Fetcher, - indexDesc client.IndexDescription, - indexFilter *mapper.Filter, -) *IndexFetcher { - return &IndexFetcher{ - docFetcher: docFetcher, - indexDesc: indexDesc, - indexFilter: indexFilter, - } -} - -func (f *IndexFetcher) Init( +// newIndexFetcher creates a new IndexFetcher. +// It can return nil, if there is no efficient way to fetch indexes with given filter conditions. +func newIndexFetcher( ctx context.Context, - identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, - acp immutable.Option[acp.ACP], + fieldsByID map[uint32]client.FieldDefinition, + indexDesc client.IndexDescription, + docFilter *mapper.Filter, col client.Collection, - fields []client.FieldDefinition, - filter *mapper.Filter, docMapper *core.DocumentMapping, - showDeleted bool, -) error { - f.resetState() + execInfo *ExecInfo, +) (*indexFetcher, error) { + f := &indexFetcher{ + ctx: ctx, + txn: txn, + col: col, + mapping: docMapper, + indexDesc: indexDesc, + fieldsByID: fieldsByID, + execInfo: execInfo, + } - f.col = col - f.doc = &encodedDocument{} - f.mapping = docMapper - f.txn = txn + fieldsToCopy := make([]mapper.Field, 0, len(indexDesc.Fields)) + for _, field := range indexDesc.Fields { + typeIndex := docMapper.FirstIndexOfName(field.Name) + indexField := mapper.Field{Index: typeIndex, Name: field.Name} + fieldsToCopy = append(fieldsToCopy, indexField) + } + for i := range fieldsToCopy { + f.indexFilter = filter.Merge(f.indexFilter, filter.CopyField(docFilter, fieldsToCopy[i])) + } for _, indexedField := range f.indexDesc.Fields { field, ok := f.col.Definition().GetFieldByName(indexedField.Name) @@ -82,158 +82,62 @@ func (f *IndexFetcher) Init( } } - f.docFields = make([]client.FieldDefinition, 0, len(fields)) -outer: - for i := range fields { - for j := range f.indexedFields { - // If the field is array, we want to keep it also for the document fetcher - // because the index only contains one array elements, not the whole array. - // The doc fetcher will fetch the whole array for us. - if fields[i].Name == f.indexedFields[j].Name && !fields[i].Kind.IsArray() { - continue outer - } - } - f.docFields = append(f.docFields, fields[i]) - } - iter, err := f.createIndexIterator() - if err != nil { - return err - } - f.indexIter = iter - - // if it turns out that we can't use the index, we need to fall back to the document fetcher - if f.indexIter == nil { - f.docFields = fields + if err != nil || iter == nil { + return nil, err } - if len(f.docFields) > 0 { - err = f.docFetcher.Init( - ctx, - identity, - f.txn, - acp, - f.col, - f.docFields, - filter, - f.mapping, - false, - ) - } - - return err + f.indexIter = iter + return f, iter.Init(ctx, txn.Datastore()) } -func (f *IndexFetcher) Start(ctx context.Context, prefixes ...keys.Walkable) error { - if f.indexIter == nil { - return f.docFetcher.Start(ctx, prefixes...) - } - return f.indexIter.Init(ctx, f.txn.Datastore()) -} +func (f *indexFetcher) NextDoc() (immutable.Option[string], error) { + f.currentDocID = immutable.None[string]() -func (f *IndexFetcher) FetchNext(ctx context.Context) (EncodedDocument, ExecInfo, error) { - if f.indexIter == nil { - return f.docFetcher.FetchNext(ctx) + res, err := f.indexIter.Next() + if err != nil || !res.foundKey { + return immutable.None[string](), err } - totalExecInfo := f.execInfo - defer func() { f.execInfo.Add(totalExecInfo) }() - f.execInfo.Reset() - for { - f.doc.Reset() - - res, err := f.indexIter.Next() - if err != nil { - return nil, ExecInfo{}, err - } - - if !res.foundKey { - return nil, f.execInfo, nil - } - - hasNilField := false - for i, indexedField := range f.indexedFields { - property := &encProperty{Desc: indexedField} - - field := res.key.Fields[i] - if field.Value.IsNil() { - hasNilField = true - } - - // Index will fetch only 1 array element. So we skip it here and let doc fetcher - // fetch the whole array. - if indexedField.Kind.IsArray() { - continue - } - - // We need to convert it to cbor bytes as this is what it will be encoded from on value retrieval. - // In the future we have to either get rid of CBOR or properly handle different encoding - // for properties in a single document. - fieldBytes, err := client.NewFieldValue(client.NONE_CRDT, field.Value).Bytes() - if err != nil { - return nil, ExecInfo{}, err - } - property.Raw = fieldBytes - - f.doc.properties[indexedField] = property - } - if f.indexDesc.Unique && !hasNilField { - f.doc.id = res.value - } else { - lastVal := res.key.Fields[len(res.key.Fields)-1].Value - if str, ok := lastVal.String(); ok { - f.doc.id = []byte(str) - } else if bytes, ok := lastVal.Bytes(); ok { - f.doc.id = bytes - } else { - return nil, ExecInfo{}, err - } - } + hasNilField := false + for i := range f.indexedFields { + hasNilField = hasNilField || res.key.Fields[i].Value.IsNil() + } - if len(f.docFields) > 0 { - targetKey := base.MakeDataStoreKeyWithCollectionAndDocID(f.col.Description(), string(f.doc.id)) - err := f.docFetcher.Start(ctx, targetKey) - if err != nil { - return nil, ExecInfo{}, err - } - encDoc, execInfo, err := f.docFetcher.FetchNext(ctx) - if err != nil { - return nil, ExecInfo{}, errors.Join(err, f.docFetcher.Close()) - } - err = f.docFetcher.Close() - if err != nil { - return nil, ExecInfo{}, err - } - f.execInfo.Add(execInfo) - if encDoc == nil { - continue - } - f.doc.MergeProperties(encDoc) + if f.indexDesc.Unique && !hasNilField { + f.currentDocID = immutable.Some(string(res.value)) + } else { + lastVal := res.key.Fields[len(res.key.Fields)-1].Value + if str, ok := lastVal.String(); ok { + f.currentDocID = immutable.Some(str) } else { - f.execInfo.DocsFetched++ + f.currentDocID = immutable.None[string]() } - return f.doc, f.execInfo, nil } + return f.currentDocID, nil } -func (f *IndexFetcher) Close() error { - if f.indexIter == nil { - return f.docFetcher.Close() +func (f *indexFetcher) GetFields() (immutable.Option[EncodedDocument], error) { + if !f.currentDocID.HasValue() { + return immutable.Option[EncodedDocument]{}, nil + } + prefix := base.MakeDataStoreKeyWithCollectionAndDocID(f.col.Description(), f.currentDocID.Value()) + prefixFetcher, err := newPrefixFetcher(f.ctx, f.txn, []keys.DataStoreKey{prefix}, f.col, + f.fieldsByID, client.Active, f.execInfo) + if err != nil { + return immutable.Option[EncodedDocument]{}, err + } + _, err = prefixFetcher.NextDoc() + if err != nil { + return immutable.Option[EncodedDocument]{}, err } - return f.indexIter.Close() + doc, err := prefixFetcher.GetFields() + return doc, errors.Join(err, prefixFetcher.Close()) } -// resetState resets the mutable state of this IndexFetcher, returning the state to how it -// was immediately after construction. -func (f *IndexFetcher) resetState() { - // WARNING: Do not reset properties set in the constructor! - - f.col = nil - f.txn = nil - f.doc = nil - f.mapping = nil - f.indexedFields = nil - f.docFields = nil - f.indexIter = nil - f.execInfo.Reset() +func (f *indexFetcher) Close() error { + if f.indexIter != nil { + return f.indexIter.Close() + } + return nil } diff --git a/internal/db/fetcher/indexer_iterators.go b/internal/db/fetcher/indexer_iterators.go index 3fba778344..39fcab006f 100644 --- a/internal/db/fetcher/indexer_iterators.go +++ b/internal/db/fetcher/indexer_iterators.go @@ -14,13 +14,13 @@ import ( "context" ds "github.com/ipfs/go-datastore" - "golang.org/x/exp/slices" "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/internal/connor" "github.com/sourcenetwork/defradb/internal/keys" + "github.com/sourcenetwork/defradb/internal/planner/filter" "github.com/sourcenetwork/defradb/internal/planner/mapper" "github.com/ipfs/go-datastore/query" @@ -42,6 +42,7 @@ const ( compOpAny = "_any" compOpAll = "_all" compOpNone = "_none" + opNot = "_not" // it's just there for composite indexes. We construct a slice of value matchers with // every matcher being responsible for a corresponding field in the index to match. // For some fields there might not be any criteria to match. For examples if you have @@ -309,7 +310,7 @@ func (iter *memorizingIndexIterator) Close() error { // newPrefixIteratorFromConditions creates a new eqPrefixIndexIterator for fetching indexed data. // It can modify the input matchers slice. -func (f *IndexFetcher) newPrefixIteratorFromConditions( +func (f *indexFetcher) newPrefixIteratorFromConditions( fieldConditions []fieldFilterCond, matchers []valueMatcher, ) (*indexPrefixIterator, error) { @@ -339,11 +340,10 @@ func (f *IndexFetcher) newPrefixIteratorFromConditions( } key := f.newIndexDataStoreKeyWithValues(keyFieldValues) - - return f.newPrefixIterator(key, matchers, &f.execInfo), nil + return f.newPrefixIterator(key, matchers, f.execInfo), nil } -func (f *IndexFetcher) newPrefixIterator( +func (f *indexFetcher) newPrefixIterator( indexKey keys.IndexDataStoreKey, matchers []valueMatcher, execInfo *ExecInfo, @@ -359,7 +359,7 @@ func (f *IndexFetcher) newPrefixIterator( // newInIndexIterator creates a new inIndexIterator for fetching indexed data. // It can modify the input matchers slice. -func (f *IndexFetcher) newInIndexIterator( +func (f *indexFetcher) newInIndexIterator( fieldConditions []fieldFilterCond, matchers []valueMatcher, ) (*inIndexIterator, error) { @@ -382,28 +382,21 @@ func (f *IndexFetcher) newInIndexIterator( } key := f.newIndexDataStoreKeyWithValues(keyFieldValues) - - iter = &eqSingleIndexIterator{ - indexKey: key, - execInfo: &f.execInfo, - } + iter = &eqSingleIndexIterator{indexKey: key, execInfo: f.execInfo} } else { indexKey := f.newIndexDataStoreKey() indexKey.Fields = []keys.IndexedField{{Descending: f.indexDesc.Fields[0].Descending}} - iter = f.newPrefixIterator(indexKey, matchers, &f.execInfo) + iter = f.newPrefixIterator(indexKey, matchers, f.execInfo) } - return &inIndexIterator{ - indexIterator: iter, - inValues: inValues, - }, nil + return &inIndexIterator{indexIterator: iter, inValues: inValues}, nil } -func (f *IndexFetcher) newIndexDataStoreKey() keys.IndexDataStoreKey { +func (f *indexFetcher) newIndexDataStoreKey() keys.IndexDataStoreKey { return keys.IndexDataStoreKey{CollectionID: f.col.Description().RootID, IndexID: f.indexDesc.ID} } -func (f *IndexFetcher) newIndexDataStoreKeyWithValues(values []client.NormalValue) keys.IndexDataStoreKey { +func (f *indexFetcher) newIndexDataStoreKeyWithValues(values []client.NormalValue) keys.IndexDataStoreKey { fields := make([]keys.IndexedField, len(values)) for i := range values { fields[i].Value = values[i] @@ -412,13 +405,13 @@ func (f *IndexFetcher) newIndexDataStoreKeyWithValues(values []client.NormalValu return keys.NewIndexDataStoreKey(f.col.Description().RootID, f.indexDesc.ID, fields) } -func (f *IndexFetcher) createIndexIterator() (indexIterator, error) { +func (f *indexFetcher) createIndexIterator() (indexIterator, error) { fieldConditions, err := f.determineFieldFilterConditions() if err != nil { return nil, err } - // this can happen if a query contains an empty condition like User(filter: {name: {}}) + // fieldConditions might be empty if a query contains an empty condition like User(filter: {name: {}}) if len(fieldConditions) == 0 { return nil, nil } @@ -428,24 +421,6 @@ func (f *IndexFetcher) createIndexIterator() (indexIterator, error) { return nil, err } - hasArray := false - for i := range fieldConditions { - if len(fieldConditions[i].arrOp) > 0 { - hasArray = true - if fieldConditions[i].arrOp == compOpNone { - matchers[i] = &invertedMatcher{matcher: matchers[i]} - } - } - } - - hasJSON := false - for i := range fieldConditions { - if fieldConditions[i].kind == client.FieldKind_NILLABLE_JSON { - hasJSON = true - break - } - } - var iter indexIterator if fieldConditions[0].op == opEq { @@ -456,7 +431,7 @@ func (f *IndexFetcher) createIndexIterator() (indexIterator, error) { } key := f.newIndexDataStoreKeyWithValues(keyFieldValues) - iter = &eqSingleIndexIterator{indexKey: key, execInfo: &f.execInfo} + iter = &eqSingleIndexIterator{indexKey: key, execInfo: f.execInfo} } else { iter, err = f.newPrefixIteratorFromConditions(fieldConditions, matchers) } @@ -464,13 +439,14 @@ func (f *IndexFetcher) createIndexIterator() (indexIterator, error) { iter, err = f.newInIndexIterator(fieldConditions, matchers) } else { key := f.newIndexDataStoreKey() + // if the first field is JSON, we want to add the JSON path prefix to scope the search if fieldConditions[0].kind == client.FieldKind_NILLABLE_JSON { key.Fields = []keys.IndexedField{{ Descending: f.indexDesc.Fields[0].Descending, Value: client.NewNormalJSON(client.MakeVoidJSON(fieldConditions[0].jsonPath)), }} } - iter, err = f.newPrefixIterator(key, matchers, &f.execInfo), nil + iter, err = f.newPrefixIterator(key, matchers, f.execInfo), nil } if err != nil { @@ -481,13 +457,23 @@ func (f *IndexFetcher) createIndexIterator() (indexIterator, error) { return nil, NewErrInvalidFilterOperator(fieldConditions[0].op) } - if hasJSON || hasArray { + if doConditionsHaveArrayOrJSON(fieldConditions) { iter = &memorizingIndexIterator{inner: iter} } return iter, nil } +func doConditionsHaveArrayOrJSON(conditions []fieldFilterCond) bool { + hasArray := false + hasJSON := false + for i := range conditions { + hasJSON = hasJSON || conditions[i].kind == client.FieldKind_NILLABLE_JSON + hasArray = hasArray || conditions[i].kind.IsArray() + } + return hasArray || hasJSON +} + type fieldFilterCond struct { op string arrOp string @@ -499,106 +485,146 @@ type fieldFilterCond struct { // determineFieldFilterConditions determines the conditions and their corresponding operation // for each indexed field. // It returns a slice of fieldFilterCond, where each element corresponds to a field in the index. -func (f *IndexFetcher) determineFieldFilterConditions() ([]fieldFilterCond, error) { +func (f *indexFetcher) determineFieldFilterConditions() ([]fieldFilterCond, error) { result := make([]fieldFilterCond, 0, len(f.indexedFields)) - for i := range f.indexedFields { - fieldInd := f.mapping.FirstIndexOfName(f.indexedFields[i].Name) - found := false - // iterate through conditions and find the one that matches the current field - for filterKey, indexFilterCond := range f.indexFilter.Conditions { - propKey, ok := filterKey.(*mapper.PropertyIndex) - if !ok || fieldInd != propKey.Index { - continue - } + // we process first the conditions that match composite index fields starting from the first one + for i := range f.indexDesc.Fields { + indexedField := f.indexedFields[i] + fieldInd := f.mapping.FirstIndexOfName(indexedField.Name) + var err error + + filter.TraverseProperties( + f.indexFilter.Conditions, + func(prop *mapper.PropertyIndex, condMap map[connor.FilterKey]any) bool { + if fieldInd != prop.Index { + return true + } - found = true - - fieldDef := f.indexedFields[slices.IndexFunc(f.indexedFields, func(f client.FieldDefinition) bool { - return int(f.ID) == fieldInd - })] - - condMap := indexFilterCond.(map[connor.FilterKey]any) - - jsonPath := client.JSONPath{} - if fieldDef.Kind == client.FieldKind_NILLABLE_JSON { - jsonPathLoop: - for { - for key, filterVal := range condMap { - prop, ok := key.(*mapper.ObjectProperty) - if !ok { - // if filter contains an array condition, we need to append index 0 to the json path - // to limit the search only to array elements - op, ok := key.(*mapper.Operator) - if ok && isArrayCondition(op.Operation) { - if op.Operation == compOpNone { - // if the array condition is _none it doesn't make sense to use index because - // values picked by the index is random guessing. For example if we have doc1 - // with array of [3, 5, 1] and doc2 with [7, 4, 8] the index first fetches - // value 1 of doc1, let it go through the filter and then fetches value 3 of doc1 - // again, skips it (because it cached doc1 id) and fetches value 4 of doc2, and - // so on until it exhaust all prefixes in ascending order. - // It might be even less effective than just scanning all documents. - return nil, nil - } - jsonPath = jsonPath.AppendIndex(0) - } - break jsonPathLoop - } - jsonPath = jsonPath.AppendProperty(prop.Name) - condMap = filterVal.(map[connor.FilterKey]any) + var jsonPath client.JSONPath + condMap, jsonPath = getNestedOperatorConditionIfJSON(indexedField, condMap) + + for key, filterVal := range condMap { + op := key.(*mapper.Operator).Operation + + // if the array condition is _none it doesn't make sense to use index because + // values picked by the index is random guessing. For example if we have doc1 + // with array of [3, 5, 1] and doc2 with [7, 4, 8] the index first fetches + // value 1 of doc1, let it go through the filter and then fetches value 3 of doc1 + // again, skips it (because it cached doc1 id) and fetches value 4 of doc2, and + // so on until it exhaust all prefixes in ascending order. + // It might be even less effective than just scanning all documents. + if op == compOpNone { + return true } - } - } - for key, filterVal := range condMap { - cond := fieldFilterCond{ - op: key.(*mapper.Operator).Operation, - jsonPath: jsonPath, - kind: f.indexedFields[i].Kind, - } + cond, err := makeFieldFilterCondition(op, jsonPath, indexedField, filterVal) - var err error - if len(jsonPath) > 0 { - err = setJSONFilterCondition(&cond, filterVal, jsonPath) - } else if filterVal == nil { - cond.val, err = client.NewNormalNil(cond.kind) - } else if !f.indexedFields[i].Kind.IsArray() { - cond.val, err = client.NewNormalValue(filterVal) - } else { - subCondMap := filterVal.(map[connor.FilterKey]any) - for subKey, subVal := range subCondMap { - if subVal == nil { - arrKind := cond.kind.(client.ScalarArrayKind) - cond.val, err = client.NewNormalNil(arrKind.SubKind()) - } else { - cond.val, err = client.NewNormalValue(subVal) - } - cond.arrOp = cond.op - cond.op = subKey.(*mapper.Operator).Operation - // the sub condition is supposed to have only 1 record - break + if err != nil { + return false } - } - if err != nil { - return nil, err + result = append(result, cond) + break } - result = append(result, cond) - break - } - break + return false + }, + // if the filter contains _not operator, we ignore the entire branch because in this + // case index will do more harm. For example if we have _not: {_eq: 5} and the index + // fetches value 5, it will skip all documents with value 5, but we need to return them. + opNot, + ) + + // if after traversing the filter for the first field we didn't find any condition that can + // be used with the index, we return nil indicating that the index can't be used. + if len(result) == 0 { + return nil, err } - if !found { + + // if traversing for the current (not first) field of the composite index didn't find any + // condition, we add a dummy that will match any value for this field. + if len(result) == i { result = append(result, fieldFilterCond{ op: opAny, val: client.NormalVoid{}, - kind: f.indexedFields[i].Kind, + kind: indexedField.Kind, }) } } return result, nil } +// makeFieldFilterCondition creates a fieldFilterCond based on the given operator and filter value on +// the given indexed field. +// If jsonPath is not empty, it means that the indexed field is a JSON field and the filter value +// should be treated as a JSON value. +func makeFieldFilterCondition( + op string, + jsonPath client.JSONPath, + indexedField client.FieldDefinition, + filterVal any, +) (fieldFilterCond, error) { + cond := fieldFilterCond{ + op: op, + jsonPath: jsonPath, + kind: indexedField.Kind, + } + + var err error + if len(jsonPath) > 0 { + err = setJSONFilterCondition(&cond, filterVal, jsonPath) + } else if filterVal == nil { + cond.val, err = client.NewNormalNil(cond.kind) + } else if !indexedField.Kind.IsArray() { + cond.val, err = client.NewNormalValue(filterVal) + } else { + subCondMap := filterVal.(map[connor.FilterKey]any) + for subKey, subVal := range subCondMap { + if subVal == nil { + arrKind := cond.kind.(client.ScalarArrayKind) + cond.val, err = client.NewNormalNil(arrKind.SubKind()) + } else { + cond.val, err = client.NewNormalValue(subVal) + } + cond.arrOp = cond.op + cond.op = subKey.(*mapper.Operator).Operation + // the sub condition is supposed to have only 1 record + break + } + } + return cond, err +} + +// getNestedOperatorConditionIfJSON traverses the filter map if the indexed field is JSON to find the +// nested operator condition and returns it along with the JSON path to the nested field. +// If the indexed field is not JSON, it returns the original condition map. +func getNestedOperatorConditionIfJSON( + indexedField client.FieldDefinition, + condMap map[connor.FilterKey]any, +) (map[connor.FilterKey]any, client.JSONPath) { + if indexedField.Kind != client.FieldKind_NILLABLE_JSON { + return condMap, client.JSONPath{} + } + var jsonPath client.JSONPath + for { + for key, filterVal := range condMap { + prop, ok := key.(*mapper.ObjectProperty) + if !ok { + // if filter contains an array condition, we need to append index 0 to the json path + // to limit the search only to array elements + op, ok := key.(*mapper.Operator) + if ok && isArrayCondition(op.Operation) { + jsonPath = jsonPath.AppendIndex(0) + } + return condMap, jsonPath + } + jsonPath = jsonPath.AppendProperty(prop.Name) + // if key is ObjectProperty it's safe to cast filterVal to map[connor.FilterKey]any + // containing either another nested ObjectProperty or Operator + condMap = filterVal.(map[connor.FilterKey]any) + } + } +} + // setJSONFilterCondition sets up the given condition struct based on the filter value and JSON path so that // it can be used to fetch the indexed data. func setJSONFilterCondition(cond *fieldFilterCond, filterVal any, jsonPath client.JSONPath) error { diff --git a/internal/db/fetcher/indexer_matchers.go b/internal/db/fetcher/indexer_matchers.go index 05214af2c1..987e7687e4 100644 --- a/internal/db/fetcher/indexer_matchers.go +++ b/internal/db/fetcher/indexer_matchers.go @@ -147,6 +147,28 @@ func (m *nilMatcher) Match(value client.NormalValue) (bool, error) { return value.IsNil() == m.matchNil, nil } +// compositeMatcher checks if the value satisfies any of the matchers +type compositeMatcher struct { + matchers []valueMatcher +} + +func newCompositeMatcher(matchers ...valueMatcher) *compositeMatcher { + return &compositeMatcher{matchers: matchers} +} + +func (m *compositeMatcher) Match(value client.NormalValue) (bool, error) { + for i := range m.matchers { + res, err := m.matchers[i].Match(value) + if err != nil { + return false, err + } + if res { + return true, nil + } + } + return false, nil +} + // checks if the index value is or is not in the given array type indexInArrayMatcher struct { inValues []client.NormalValue @@ -241,26 +263,13 @@ type anyMatcher struct{} func (m *anyMatcher) Match(client.NormalValue) (bool, error) { return true, nil } -// invertedMatcher inverts the result of the inner matcher. -type invertedMatcher struct { - matcher valueMatcher -} - -func (m *invertedMatcher) Match(val client.NormalValue) (bool, error) { - res, err := m.matcher.Match(val) - if err != nil { - return false, err - } - return !res, nil -} - -type jsonComparableMatcher[T comparable] struct { +type jsonComparingMatcher[T comparable] struct { value T getValueFunc func(client.JSON) (T, bool) evalFunc func(T, T) bool } -func (m *jsonComparableMatcher[T]) Match(value client.NormalValue) (bool, error) { +func (m *jsonComparingMatcher[T]) Match(value client.NormalValue) (bool, error) { if jsonVal, ok := value.JSON(); ok { if val, ok := m.getValueFunc(jsonVal); ok { return m.evalFunc(val, m.value), nil @@ -270,6 +279,20 @@ func (m *jsonComparableMatcher[T]) Match(value client.NormalValue) (bool, error) return false, NewErrUnexpectedTypeValue[float64](value) } +// jsonTypeMatcher checks if the value is of the expected type +type jsonTypeMatcher[T comparable] struct { + getValueFunc func(client.JSON) (T, bool) + shouldMatch bool +} + +func (m *jsonTypeMatcher[T]) Match(value client.NormalValue) (bool, error) { + if jsonVal, ok := value.JSON(); ok { + _, ok := m.getValueFunc(jsonVal) + return ok == m.shouldMatch, nil + } + return false, NewErrUnexpectedTypeValue[float64](value) +} + type jsonBoolMatcher struct { value bool isEq bool @@ -287,7 +310,6 @@ func (m *jsonBoolMatcher) Match(value client.NormalValue) (bool, error) { } type jsonNullMatcher struct { - // TODO: _ne null is not handled yet matchNull bool } @@ -303,51 +325,13 @@ func createValueMatcher(condition *fieldFilterCond) (valueMatcher, error) { return &anyMatcher{}, nil } - // TODO: test json null if condition.val.IsNil() { return &nilMatcher{matchNil: condition.op == opEq}, nil } switch condition.op { case opEq, opGt, opGe, opLt, opLe, opNe: - if v, ok := condition.val.Int(); ok { - return &intMatcher{value: v, evalFunc: getCompareValsFunc[int64](condition.op)}, nil - } - if v, ok := condition.val.Float(); ok { - return &floatMatcher{value: v, evalFunc: getCompareValsFunc[float64](condition.op)}, nil - } - if v, ok := condition.val.String(); ok { - return &stringMatcher{value: v, evalFunc: getCompareValsFunc[string](condition.op)}, nil - } - if v, ok := condition.val.Time(); ok { - return &timeMatcher{value: v, op: condition.op}, nil - } - if v, ok := condition.val.Bool(); ok { - return &boolMatcher{value: v, isEq: condition.op == opEq}, nil - } - if v, ok := condition.val.JSON(); ok { - if jsonVal, ok := v.Number(); ok { - return &jsonComparableMatcher[float64]{ - value: jsonVal, - getValueFunc: func(j client.JSON) (float64, bool) { return j.Number() }, - evalFunc: getCompareValsFunc[float64](condition.op), - }, nil - } - if jsonVal, ok := v.String(); ok { - return &jsonComparableMatcher[string]{ - value: jsonVal, - getValueFunc: func(j client.JSON) (string, bool) { return j.String() }, - evalFunc: getCompareValsFunc[string](condition.op), - }, nil - } - if jsonVal, ok := v.Bool(); ok { - // TODO: test bool not equal - return &jsonBoolMatcher{value: jsonVal, isEq: condition.op == opEq}, nil - } - if v.IsNull() { - return &jsonNullMatcher{matchNull: condition.op == opEq}, nil - } - } + return createComparingMatcher(condition), nil case opIn, opNin: inVals, err := client.ToArrayOfNormalValues(condition.val) if err != nil { @@ -369,6 +353,85 @@ func createValueMatcher(condition *fieldFilterCond) (valueMatcher, error) { return nil, NewErrInvalidFilterOperator(condition.op) } +func createComparingMatcher(condition *fieldFilterCond) valueMatcher { + // JSON type needs a special handling if the op is _ne, because _ne should check also + // difference of types + if v, ok := condition.val.JSON(); ok { + // we have a separate branch for null matcher because default matching behavior + // is what we need: for filter `_ne: null` it will match all non-null values + if v.IsNull() { + return &jsonNullMatcher{matchNull: condition.op == opEq} + } + + if condition.op != opNe { + return createJSONComparingMatcher(v, condition.op) + } + + // _ne filter on JSON fields should also accept values of different types + var typeMatcher valueMatcher + if _, ok := v.Number(); ok { + typeMatcher = &jsonTypeMatcher[float64]{ + getValueFunc: func(j client.JSON) (float64, bool) { return j.Number() }, + } + } else if _, ok := v.String(); ok { + typeMatcher = &jsonTypeMatcher[string]{ + getValueFunc: func(j client.JSON) (string, bool) { return j.String() }, + } + } else if _, ok := v.Bool(); ok { + typeMatcher = &jsonTypeMatcher[bool]{ + getValueFunc: func(j client.JSON) (bool, bool) { return j.Bool() }, + } + } + return newCompositeMatcher(typeMatcher, createJSONComparingMatcher(v, condition.op)) + } + + matcher := createScalarComparingMatcher(condition) + + // for _ne filter on regular (non-JSON) fields the index should also accept nil values + // there won't be `_ne: null` because nil check is done before this function is called + if condition.op == opNe { + matcher = newCompositeMatcher(&nilMatcher{matchNil: true}, matcher) + } + + return matcher +} + +// createJSONComparingMatcher creates a matcher for JSON values +func createJSONComparingMatcher(val client.JSON, op string) valueMatcher { + if jsonVal, ok := val.Number(); ok { + return &jsonComparingMatcher[float64]{ + value: jsonVal, + getValueFunc: func(j client.JSON) (float64, bool) { return j.Number() }, + evalFunc: getCompareValsFunc[float64](op), + } + } else if jsonVal, ok := val.String(); ok { + return &jsonComparingMatcher[string]{ + value: jsonVal, + getValueFunc: func(j client.JSON) (string, bool) { return j.String() }, + evalFunc: getCompareValsFunc[string](op), + } + } else if jsonVal, ok := val.Bool(); ok { + return &jsonBoolMatcher{value: jsonVal, isEq: op == opEq} + } + return nil +} + +// createScalarComparingMatcher creates a matcher for scalar values (int, float, string, time, bool) +func createScalarComparingMatcher(condition *fieldFilterCond) valueMatcher { + if v, ok := condition.val.Int(); ok { + return &intMatcher{value: v, evalFunc: getCompareValsFunc[int64](condition.op)} + } else if v, ok := condition.val.Float(); ok { + return &floatMatcher{value: v, evalFunc: getCompareValsFunc[float64](condition.op)} + } else if v, ok := condition.val.String(); ok { + return &stringMatcher{value: v, evalFunc: getCompareValsFunc[string](condition.op)} + } else if v, ok := condition.val.Time(); ok { + return &timeMatcher{value: v, op: condition.op} + } else if v, ok := condition.val.Bool(); ok { + return &boolMatcher{value: v, isEq: condition.op == opEq} + } + return nil +} + func extractStringFromNormalValue(val client.NormalValue) (string, error) { strVal, ok := val.String() if !ok { diff --git a/internal/db/fetcher/mocks/fetcher.go b/internal/db/fetcher/mocks/fetcher.go index 723403723f..5306b484f9 100644 --- a/internal/db/fetcher/mocks/fetcher.go +++ b/internal/db/fetcher/mocks/fetcher.go @@ -148,17 +148,17 @@ func (_c *Fetcher_FetchNext_Call) RunAndReturn(run func(context.Context) (fetche return _c } -// Init provides a mock function with given fields: ctx, _a1, txn, _a3, col, fields, filter, docmapper, showDeleted -func (_m *Fetcher) Init(ctx context.Context, _a1 immutable.Option[identity.Identity], txn datastore.Txn, _a3 immutable.Option[acp.ACP], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, docmapper *core.DocumentMapping, showDeleted bool) error { - ret := _m.Called(ctx, _a1, txn, _a3, col, fields, filter, docmapper, showDeleted) +// Init provides a mock function with given fields: ctx, _a1, txn, _a3, index, col, fields, filter, docmapper, showDeleted +func (_m *Fetcher) Init(ctx context.Context, _a1 immutable.Option[identity.Identity], txn datastore.Txn, _a3 immutable.Option[acp.ACP], index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, docmapper *core.DocumentMapping, showDeleted bool) error { + ret := _m.Called(ctx, _a1, txn, _a3, index, col, fields, filter, docmapper, showDeleted) if len(ret) == 0 { panic("no return value specified for Init") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, immutable.Option[identity.Identity], datastore.Txn, immutable.Option[acp.ACP], client.Collection, []client.FieldDefinition, *mapper.Filter, *core.DocumentMapping, bool) error); ok { - r0 = rf(ctx, _a1, txn, _a3, col, fields, filter, docmapper, showDeleted) + if rf, ok := ret.Get(0).(func(context.Context, immutable.Option[identity.Identity], datastore.Txn, immutable.Option[acp.ACP], immutable.Option[client.IndexDescription], client.Collection, []client.FieldDefinition, *mapper.Filter, *core.DocumentMapping, bool) error); ok { + r0 = rf(ctx, _a1, txn, _a3, index, col, fields, filter, docmapper, showDeleted) } else { r0 = ret.Error(0) } @@ -176,18 +176,19 @@ type Fetcher_Init_Call struct { // - _a1 immutable.Option[identity.Identity] // - txn datastore.Txn // - _a3 immutable.Option[acp.ACP] +// - index immutable.Option[client.IndexDescription] // - col client.Collection // - fields []client.FieldDefinition // - filter *mapper.Filter // - docmapper *core.DocumentMapping // - showDeleted bool -func (_e *Fetcher_Expecter) Init(ctx interface{}, _a1 interface{}, txn interface{}, _a3 interface{}, col interface{}, fields interface{}, filter interface{}, docmapper interface{}, showDeleted interface{}) *Fetcher_Init_Call { - return &Fetcher_Init_Call{Call: _e.mock.On("Init", ctx, _a1, txn, _a3, col, fields, filter, docmapper, showDeleted)} +func (_e *Fetcher_Expecter) Init(ctx interface{}, _a1 interface{}, txn interface{}, _a3 interface{}, index interface{}, col interface{}, fields interface{}, filter interface{}, docmapper interface{}, showDeleted interface{}) *Fetcher_Init_Call { + return &Fetcher_Init_Call{Call: _e.mock.On("Init", ctx, _a1, txn, _a3, index, col, fields, filter, docmapper, showDeleted)} } -func (_c *Fetcher_Init_Call) Run(run func(ctx context.Context, _a1 immutable.Option[identity.Identity], txn datastore.Txn, _a3 immutable.Option[acp.ACP], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, docmapper *core.DocumentMapping, showDeleted bool)) *Fetcher_Init_Call { +func (_c *Fetcher_Init_Call) Run(run func(ctx context.Context, _a1 immutable.Option[identity.Identity], txn datastore.Txn, _a3 immutable.Option[acp.ACP], index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, docmapper *core.DocumentMapping, showDeleted bool)) *Fetcher_Init_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(immutable.Option[identity.Identity]), args[2].(datastore.Txn), args[3].(immutable.Option[acp.ACP]), args[4].(client.Collection), args[5].([]client.FieldDefinition), args[6].(*mapper.Filter), args[7].(*core.DocumentMapping), args[8].(bool)) + run(args[0].(context.Context), args[1].(immutable.Option[identity.Identity]), args[2].(datastore.Txn), args[3].(immutable.Option[acp.ACP]), args[4].(immutable.Option[client.IndexDescription]), args[5].(client.Collection), args[6].([]client.FieldDefinition), args[7].(*mapper.Filter), args[8].(*core.DocumentMapping), args[9].(bool)) }) return _c } @@ -197,7 +198,7 @@ func (_c *Fetcher_Init_Call) Return(_a0 error) *Fetcher_Init_Call { return _c } -func (_c *Fetcher_Init_Call) RunAndReturn(run func(context.Context, immutable.Option[identity.Identity], datastore.Txn, immutable.Option[acp.ACP], client.Collection, []client.FieldDefinition, *mapper.Filter, *core.DocumentMapping, bool) error) *Fetcher_Init_Call { +func (_c *Fetcher_Init_Call) RunAndReturn(run func(context.Context, immutable.Option[identity.Identity], datastore.Txn, immutable.Option[acp.ACP], immutable.Option[client.IndexDescription], client.Collection, []client.FieldDefinition, *mapper.Filter, *core.DocumentMapping, bool) error) *Fetcher_Init_Call { _c.Call.Return(run) return _c } diff --git a/internal/db/fetcher/mocks/utils.go b/internal/db/fetcher/mocks/utils.go index d12f4fdd98..524c46fc9e 100644 --- a/internal/db/fetcher/mocks/utils.go +++ b/internal/db/fetcher/mocks/utils.go @@ -28,6 +28,7 @@ func NewStubbedFetcher(t *testing.T) *Fetcher { mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Maybe().Return(nil) f.EXPECT().Start(mock.Anything, mock.Anything).Maybe().Return(nil) f.EXPECT().FetchNext(mock.Anything).Maybe().Return(nil, nil) diff --git a/internal/db/fetcher/versioned.go b/internal/db/fetcher/versioned.go index de829514ba..4fdb016b87 100644 --- a/internal/db/fetcher/versioned.go +++ b/internal/db/fetcher/versioned.go @@ -102,6 +102,7 @@ func (vf *VersionedFetcher) Init( identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, acp immutable.Option[acp.ACP], + index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, @@ -136,6 +137,7 @@ func (vf *VersionedFetcher) Init( identity, vf.store, acp, + index, col, fields, filter, diff --git a/internal/db/fetcher/wrapper.go b/internal/db/fetcher/wrapper.go index 6b55f86a1f..845ecf68ce 100644 --- a/internal/db/fetcher/wrapper.go +++ b/internal/db/fetcher/wrapper.go @@ -29,13 +29,14 @@ import ( // and the newer [fetcher] interface. type wrappingFetcher struct { fetcher fetcher - execInfo *ExecInfo + execInfo ExecInfo - // The below properties are only held in state in order to temporarily adhear to the [Fetcher] + // The below properties are only held in state in order to temporarily adhere to the [Fetcher] // interface. They can be remove from state once the [Fetcher] interface is cleaned up. identity immutable.Option[acpIdentity.Identity] txn datastore.Txn acp immutable.Option[acp.ACP] + index immutable.Option[client.IndexDescription] col client.Collection fields []client.FieldDefinition filter *mapper.Filter @@ -54,6 +55,7 @@ func (f *wrappingFetcher) Init( identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, acp immutable.Option[acp.ACP], + index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, @@ -63,6 +65,7 @@ func (f *wrappingFetcher) Init( f.identity = identity f.txn = txn f.acp = acp + f.index = index f.col = col f.fields = fields f.filter = filter @@ -90,7 +93,7 @@ func (f *wrappingFetcher) Start(ctx context.Context, prefixes ...keys.Walkable) if f.filter != nil && len(f.fields) > 0 { conditions := f.filter.ToMap(f.docMapper) - parsedfilterFields, err := parser.ParseFilterFieldsForDescription(conditions, f.col.Definition()) + parsedFilterFields, err := parser.ParseFilterFieldsForDescription(conditions, f.col.Definition()) if err != nil { return err } @@ -100,7 +103,7 @@ func (f *wrappingFetcher) Start(ctx context.Context, prefixes ...keys.Walkable) existingFields[field.ID] = struct{}{} } - for _, field := range parsedfilterFields { + for _, field := range parsedFilterFields { if _, ok := existingFields[field.ID]; !ok { f.fields = append(f.fields, field) } @@ -117,17 +120,31 @@ func (f *wrappingFetcher) Start(ctx context.Context, prefixes ...keys.Walkable) fieldsByID[uint32(field.ID)] = field } - var execInfo ExecInfo - f.execInfo = &execInfo + f.execInfo.Reset() var top fetcher - top, err = newPrefixFetcher(ctx, f.txn, dsPrefixes, f.col, fieldsByID, client.Active, &execInfo) - if err != nil { - return err + if f.index.HasValue() { + indexFetcher, err := newIndexFetcher(ctx, f.txn, fieldsByID, f.index.Value(), f.filter, f.col, + f.docMapper, &f.execInfo) + if err != nil { + return err + } + if indexFetcher != nil { + top = indexFetcher + } + } + + // the index fetcher might not have been created if there is no efficient way to use fetch indexes + // with given filter conditions. In this case we fall back to the prefix fetcher + if top == nil { + top, err = newPrefixFetcher(ctx, f.txn, dsPrefixes, f.col, fieldsByID, client.Active, &f.execInfo) + if err != nil { + return err + } } if f.showDeleted { - deletedFetcher, err := newPrefixFetcher(ctx, f.txn, dsPrefixes, f.col, fieldsByID, client.Deleted, &execInfo) + deletedFetcher, err := newPrefixFetcher(ctx, f.txn, dsPrefixes, f.col, fieldsByID, client.Deleted, &f.execInfo) if err != nil { return err } @@ -148,31 +165,29 @@ func (f *wrappingFetcher) Start(ctx context.Context, prefixes ...keys.Walkable) } func (f *wrappingFetcher) FetchNext(ctx context.Context) (EncodedDocument, ExecInfo, error) { - docID, err := f.fetcher.NextDoc() - if err != nil { - return nil, ExecInfo{}, err - } - - if !docID.HasValue() { - execInfo := *f.execInfo - f.execInfo.Reset() + f.execInfo.Reset() - return nil, execInfo, nil - } + for { + docID, err := f.fetcher.NextDoc() + if err != nil { + return nil, ExecInfo{}, err + } - doc, err := f.fetcher.GetFields() - if err != nil { - return nil, ExecInfo{}, err - } + if !docID.HasValue() { + return nil, f.execInfo, nil + } - if !doc.HasValue() { - return f.FetchNext(ctx) - } + doc, err := f.fetcher.GetFields() + if err != nil { + return nil, ExecInfo{}, err + } - execInfo := *f.execInfo - f.execInfo.Reset() + if !doc.HasValue() { + continue + } - return doc.Value(), execInfo, nil + return doc.Value(), f.execInfo, nil + } } func (f *wrappingFetcher) Close() error { diff --git a/internal/db/indexed_docs_test.go b/internal/db/indexed_docs_test.go index e436f30b38..ad91944614 100644 --- a/internal/db/indexed_docs_test.go +++ b/internal/db/indexed_docs_test.go @@ -630,6 +630,7 @@ func TestNonUniqueCreate_IfUponIndexingExistingDocsFetcherFails_ReturnError(t *t mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Unset() f.EXPECT().Init( mock.Anything, @@ -641,6 +642,7 @@ func TestNonUniqueCreate_IfUponIndexingExistingDocsFetcherFails_ReturnError(t *t mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Return(testError) f.EXPECT().Close().Unset() f.EXPECT().Close().Return(nil) @@ -861,6 +863,7 @@ func TestNonUniqueUpdate_IfFetcherFails_ReturnError(t *testing.T) { mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Unset() f.EXPECT().Init( mock.Anything, @@ -872,6 +875,7 @@ func TestNonUniqueUpdate_IfFetcherFails_ReturnError(t *testing.T) { mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Return(testError) f.EXPECT().Close().Unset() f.EXPECT().Close().Return(nil) @@ -980,6 +984,7 @@ func TestNonUniqueUpdate_ShouldPassToFetcherOnlyRelevantFields(t *testing.T) { mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ).Unset() f.EXPECT().Init( mock.Anything, @@ -991,12 +996,14 @@ func TestNonUniqueUpdate_ShouldPassToFetcherOnlyRelevantFields(t *testing.T) { mock.Anything, mock.Anything, mock.Anything, + mock.Anything, ). RunAndReturn(func( ctx context.Context, identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, acp immutable.Option[acp.ACP], + index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, diff --git a/internal/lens/fetcher.go b/internal/lens/fetcher.go index e5c1f3f8e5..d8f533e96f 100644 --- a/internal/lens/fetcher.go +++ b/internal/lens/fetcher.go @@ -66,6 +66,7 @@ func (f *lensedFetcher) Init( identity immutable.Option[acpIdentity.Identity], txn datastore.Txn, acp immutable.Option[acp.ACP], + index immutable.Option[client.IndexDescription], col client.Collection, fields []client.FieldDefinition, filter *mapper.Filter, @@ -117,6 +118,7 @@ historyLoop: identity, txn, acp, + index, col, innerFetcherFields, filter, diff --git a/internal/lens/registry.go b/internal/lens/registry.go index c0fc87a14f..cba0d2a90a 100644 --- a/internal/lens/registry.go +++ b/internal/lens/registry.go @@ -144,7 +144,7 @@ func (r *lensRegistry) getCtx(txn datastore.Txn, readonly bool) *txnContext { } func (r *lensRegistry) setMigration( - ctx context.Context, + _ context.Context, txnCtx *txnContext, collectionID uint32, cfg model.Lens, @@ -182,7 +182,7 @@ func (r *lensRegistry) setMigration( } func (r *lensRegistry) cachePool( - txn datastore.Txn, + _ datastore.Txn, target map[uint32]*lensPool, cfg model.Lens, collectionID uint32, diff --git a/internal/planner/filter/traverse.go b/internal/planner/filter/traverse.go new file mode 100644 index 0000000000..0f837efe67 --- /dev/null +++ b/internal/planner/filter/traverse.go @@ -0,0 +1,140 @@ +// Copyright 2025 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 filter + +import ( + "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/internal/connor" + "github.com/sourcenetwork/defradb/internal/planner/mapper" +) + +// TraverseFields walks through a filter condition tree and calls the provided function f +// for each leaf node (field value) encountered. The function f receives the path to the field +// (as a string slice) and its value. If f returns false, traversal stops immediately. +// +// The path parameter in f represents the nested field names leading to the value, excluding +// operator keys (those starting with '_'). For example, given the filter: +// +// { +// "author": { +// "books": { +// "title": {"_eq": "Sample"} +// } +// } +// } +// +// The callback would receive path=["author", "books", "title"] and value="Sample" +func TraverseFields(conditions map[string]any, f func([]string, any) bool) { + traverseFields(nil, "", conditions, f) +} + +func traverseFields(path []string, key string, value any, f func([]string, any) bool) bool { + isKeyOp := func(k string) bool { return len(k) > 0 && k[0] == '_' && k != request.DocIDFieldName } + isOpComplex := func(k string) bool { + switch k { + // all these ops should have a map or an array as value and can not have a single value + case request.AliasFieldName, request.FilterOpOr, request.FilterOpAnd, request.FilterOpNot: + return true + } + return false + } + switch t := value.(type) { + case map[string]any: + for key, value := range t { + if isKeyOp(key) { + if !traverseFields(path, key, value, f) { + return false + } + } else { + newPath := make([]string, len(path), len(path)+1) + copy(newPath, path) + newPath = append(newPath, key) + if !traverseFields(newPath, key, value, f) { + return false + } + } + } + case []any: + for _, v := range t { + if !traverseFields(path, "", v, f) { + return false + } + } + default: + if isKeyOp(key) && isOpComplex(key) { + return false + } + return f(path, value) + } + return true +} + +// TraverseProperties walks through a mapper filter tree and calls the provided function f +// for each PropertyIndex node encountered. Unlike TraverseFields, this function works with +// the internal filter representation using mapper.PropertyIndex and connor.FilterKey types. +// +// The function f receives: +// - The property index node being visited +// - A map of its conditions +// +// If f returns false, traversal stops immediately. +func TraverseProperties( + conditions map[connor.FilterKey]any, + f func(*mapper.PropertyIndex, map[connor.FilterKey]any) bool, + skipOps ...string, +) { + traverseProperties(nil, conditions, f, skipOps) +} + +func traverseProperties( + path []string, + conditions map[connor.FilterKey]any, + f func(*mapper.PropertyIndex, map[connor.FilterKey]any) bool, + skipOps []string, +) bool { + for filterKey, cond := range conditions { + switch t := filterKey.(type) { + case *mapper.PropertyIndex: + if condMap, ok := cond.(map[connor.FilterKey]any); ok { + if !f(t, condMap) { + return false + } + } + case *mapper.Operator: + // Skip this operator if it's in the ignore list + shouldIgnore := false + for _, ignore := range skipOps { + if t.Operation == ignore { + shouldIgnore = true + break + } + } + if shouldIgnore { + continue + } + + switch condVal := cond.(type) { + case map[connor.FilterKey]any: + if !traverseProperties(path, condVal, f, skipOps) { + return false + } + case []any: + for _, elem := range condVal { + if elemMap, ok := elem.(map[connor.FilterKey]any); ok { + if !traverseProperties(path, elemMap, f, skipOps) { + return false + } + } + } + } + } + } + return true +} diff --git a/internal/planner/filter/traverse_test.go b/internal/planner/filter/traverse_test.go new file mode 100644 index 0000000000..b5dfb23c6e --- /dev/null +++ b/internal/planner/filter/traverse_test.go @@ -0,0 +1,494 @@ +// Copyright 2025 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 filter + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/internal/connor" + "github.com/sourcenetwork/defradb/internal/planner/mapper" +) + +func TestTraverseFields(t *testing.T) { + tests := []struct { + name string + input map[string]any + expectedPaths [][]string + expectedVals []any + }{ + { + name: "simple field", + input: map[string]any{ + "name": map[string]any{ + "_eq": "John", + }, + }, + expectedPaths: [][]string{{"name"}}, + expectedVals: []any{"John"}, + }, + { + name: "multiple fields", + input: map[string]any{ + "name": map[string]any{"_eq": "John"}, + "age": map[string]any{"_gt": 25}, + }, + expectedPaths: [][]string{{"name"}, {"age"}}, + expectedVals: []any{"John", 25}, + }, + { + name: "nested fields", + input: map[string]any{ + "author": map[string]any{ + "books": map[string]any{ + "title": map[string]any{ + "_eq": "Sample Book", + }, + }, + }, + }, + expectedPaths: [][]string{{"author", "books", "title"}}, + expectedVals: []any{"Sample Book"}, + }, + { + name: "with _or operator", + input: map[string]any{ + request.FilterOpOr: []any{ + map[string]any{ + "name": map[string]any{"_eq": "John"}, + }, + map[string]any{ + "age": map[string]any{"_gt": 30}, + }, + }, + }, + expectedPaths: [][]string{{"name"}, {"age"}}, + expectedVals: []any{"John", 30}, + }, + { + name: "with _or operator with nil value", + input: map[string]any{ + request.FilterOpOr: nil, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _or operator with empty array value", + input: map[string]any{ + request.FilterOpOr: []any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _or operator with empty map value", + input: map[string]any{ + request.FilterOpOr: map[string]any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _or operator with invalid value", + input: map[string]any{ + request.FilterOpOr: 1, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _and operator", + input: map[string]any{ + request.FilterOpAnd: []any{ + map[string]any{ + "name": map[string]any{"_eq": "John"}, + }, + map[string]any{ + "age": map[string]any{"_gt": 30}, + }, + }, + }, + expectedPaths: [][]string{{"name"}, {"age"}}, + expectedVals: []any{"John", 30}, + }, + { + name: "with _and operator with nil value", + input: map[string]any{ + request.FilterOpAnd: nil, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _and operator with empty array value", + input: map[string]any{ + request.FilterOpAnd: []any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _and operator with empty map value", + input: map[string]any{ + request.FilterOpAnd: map[string]any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _and operator with invalid value", + input: map[string]any{ + request.FilterOpAnd: 1, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _not operator", + input: map[string]any{ + request.FilterOpNot: []any{ + map[string]any{ + "name": map[string]any{"_eq": "John"}, + }, + }, + }, + expectedPaths: [][]string{{"name"}}, + expectedVals: []any{"John"}, + }, + { + name: "with _not operator with nil value", + input: map[string]any{ + request.FilterOpNot: nil, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _not operator with empty array value", + input: map[string]any{ + request.FilterOpNot: []any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _not operator with empty map value", + input: map[string]any{ + request.FilterOpNot: map[string]any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with _not operator with invalid value", + input: map[string]any{ + request.FilterOpNot: 1, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with alias operator with nil value", + input: map[string]any{ + request.AliasFieldName: nil, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with alias operator", + input: map[string]any{ + request.AliasFieldName: map[string]any{ + "age": map[string]any{"_eq": 30}, + }, + }, + expectedPaths: [][]string{{"age"}}, + expectedVals: []any{30}, + }, + { + name: "with empty alias operator", + input: map[string]any{ + request.AliasFieldName: map[string]any{}, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + { + name: "with alias operator of invalid type", + input: map[string]any{ + request.AliasFieldName: 1, + }, + expectedPaths: [][]string{}, + expectedVals: []any{}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var actualPaths [][]string + var actualVals []any + + TraverseFields(tt.input, func(path []string, val any) bool { + pathCopy := make([]string, len(path)) + copy(pathCopy, path) + actualPaths = append(actualPaths, pathCopy) + actualVals = append(actualVals, val) + return true // continue traversal + }) + + assert.ElementsMatch(t, tt.expectedPaths, actualPaths) + assert.ElementsMatch(t, tt.expectedVals, actualVals) + }) + } +} + +func TestTraverseFieldsEarlyExit(t *testing.T) { + tests := []struct { + name string + input map[string]any + expectedCount int + exitAfter int + }{ + { + name: "exit in flat fields", + input: map[string]any{ + "name": map[string]any{"_eq": "John"}, + "age": map[string]any{"_gt": 25}, + "city": map[string]any{"_eq": "New York"}, + }, + expectedCount: 2, + exitAfter: 2, + }, + { + name: "exit in nested fields", + input: map[string]any{ + "author": map[string]any{ + "name": map[string]any{"_eq": "John"}, + "books": map[string]any{ + "title": map[string]any{"_eq": "Book 1"}, + "year": map[string]any{"_gt": 2000}, + }, + }, + }, + expectedCount: 1, + exitAfter: 1, + }, + { + name: "exit in array operator", + input: map[string]any{ + "_or": []any{ + map[string]any{ + "name": map[string]any{"_eq": "John"}, + }, + map[string]any{ + "age": map[string]any{"_gt": 30}, + }, + map[string]any{ + "city": map[string]any{"_eq": "Paris"}, + }, + }, + }, + expectedCount: 2, + exitAfter: 2, + }, + { + name: "exit in mixed operators", + input: map[string]any{ + "_and": []any{ + map[string]any{ + "name": map[string]any{"_eq": "John"}, + }, + map[string]any{ + "_or": []any{ + map[string]any{"age": map[string]any{"_gt": 30}}, + map[string]any{"city": map[string]any{"_eq": "Paris"}}, + }, + }, + }, + }, + expectedCount: 1, + exitAfter: 1, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var actualPaths [][]string + var actualVals []any + + TraverseFields(tt.input, func(path []string, val any) bool { + pathCopy := make([]string, len(path)) + copy(pathCopy, path) + actualPaths = append(actualPaths, pathCopy) + actualVals = append(actualVals, val) + return len(actualPaths) < tt.exitAfter + }) + + assert.Equal(t, tt.expectedCount, len(actualPaths), + "should have stopped after %d fields", tt.expectedCount) + assert.Equal(t, tt.expectedCount, len(actualVals), + "should have stopped after %d values", tt.expectedCount) + }) + } +} + +func TestTraverseProperties(t *testing.T) { + tests := []struct { + name string + input map[connor.FilterKey]any + expectedProps []int + expectedValues map[int]any + }{ + { + name: "simple property", + input: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + }, + expectedProps: []int{1}, + expectedValues: map[int]any{1: "John"}, + }, + { + name: "multiple properties", + input: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_gt"}: 25, + }, + }, + expectedProps: []int{1, 2}, + expectedValues: map[int]any{1: "John", 2: 25}, + }, + { + name: "nested in operator", + input: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_or"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_gt"}: 25, + }, + }, + }, + expectedProps: []int{1, 2}, + expectedValues: map[int]any{1: "John", 2: 25}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var actualProps []int + actualValues := make(map[int]any) + + TraverseProperties(tt.input, func(prop *mapper.PropertyIndex, val map[connor.FilterKey]any) bool { + actualProps = append(actualProps, prop.Index) + // Extract the actual value from the operator map + for _, v := range val { + actualValues[prop.Index] = v + break // We only expect one operator per property in our test cases + } + return true + }) + + assert.ElementsMatch(t, tt.expectedProps, actualProps) + assert.Equal(t, tt.expectedValues, actualValues) + }) + } +} + +func TestTraverseProperties_EarlyExit(t *testing.T) { + input := map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_and"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_gt"}: 25, + }, + &mapper.PropertyIndex{Index: 3}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "New York", + }, + }, + } + + var visitCount int + TraverseProperties(input, func(prop *mapper.PropertyIndex, val map[connor.FilterKey]any) bool { + visitCount++ + return visitCount < 2 // Stop after visiting 2 properties + }) + + assert.Equal(t, 2, visitCount, "should have stopped after visiting 2 properties") +} + +func TestTraversePropertiesWithIgnoreNodes(t *testing.T) { + tests := []struct { + name string + conditions map[connor.FilterKey]any + skipOps []string + expected []int + }{ + { + name: "ignore _not operator", + conditions: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_not"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + }, + &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_gt"}: 18, + }, + }, + skipOps: []string{"_not"}, + expected: []int{2}, + }, + { + name: "ignore multiple operators", + conditions: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_not"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: "John", + }, + }, + &mapper.Operator{Operation: "_and"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_eq"}: true, + }, + }, + &mapper.Operator{Operation: "_or"}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 3}: map[connor.FilterKey]any{ + &mapper.Operator{Operation: "_gt"}: 18, + }, + }, + }, + skipOps: []string{"_not", "_or"}, + expected: []int{2}, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var visited []int + TraverseProperties(tt.conditions, func(p *mapper.PropertyIndex, conditions map[connor.FilterKey]any) bool { + visited = append(visited, p.Index) + return true + }, tt.skipOps...) + + assert.ElementsMatch(t, tt.expected, visited) + }) + } +} diff --git a/internal/planner/mapper/targetable.go b/internal/planner/mapper/targetable.go index e25c8b03f5..695dfede6d 100644 --- a/internal/planner/mapper/targetable.go +++ b/internal/planner/mapper/targetable.go @@ -30,11 +30,11 @@ type PropertyIndex struct { Index int } -func (k *PropertyIndex) PropertyAndOperator(data any, defaultOp string) (any, string, error) { +func (k *PropertyIndex) PropertyAndOperator(data any, defaultOp string) (connor.KeyResult, error) { if data == nil { - return nil, defaultOp, nil + return connor.KeyResult{Data: nil, Operator: defaultOp}, nil } - return data.(core.Doc).Fields[k.Index], defaultOp, nil + return connor.KeyResult{Data: data.(core.Doc).Fields[k.Index], Operator: defaultOp}, nil } func (k *PropertyIndex) Equal(other connor.FilterKey) bool { @@ -52,8 +52,8 @@ type Operator struct { Operation string } -func (k *Operator) PropertyAndOperator(data any, defaultOp string) (any, string, error) { - return data, k.Operation, nil +func (k *Operator) PropertyAndOperator(data any, defaultOp string) (connor.KeyResult, error) { + return connor.KeyResult{Data: data, Operator: k.Operation}, nil } func (k *Operator) Equal(other connor.FilterKey) bool { @@ -72,15 +72,16 @@ type ObjectProperty struct { Name string } -func (k *ObjectProperty) PropertyAndOperator(data any, defaultOp string) (any, string, error) { +func (k *ObjectProperty) PropertyAndOperator(data any, defaultOp string) (connor.KeyResult, error) { if data == nil { - return nil, defaultOp, nil + return connor.KeyResult{Operator: defaultOp}, nil } docMap, ok := data.(map[string]any) if !ok { - return nil, defaultOp, NewErrFieldOrAliasNotFound(k.Name) + return connor.KeyResult{Operator: defaultOp}, NewErrFieldOrAliasNotFound(k.Name) } - return docMap[k.Name], defaultOp, nil + prop, hasProp := docMap[k.Name] + return connor.KeyResult{Data: prop, MissProp: !hasProp, Operator: defaultOp}, nil } func (k *ObjectProperty) Equal(other connor.FilterKey) bool { diff --git a/internal/planner/planner.go b/internal/planner/planner.go index 8390c6d5a5..8d5c7cf052 100644 --- a/internal/planner/planner.go +++ b/internal/planner/planner.go @@ -309,11 +309,9 @@ func (p *Planner) tryOptimizeJoinDirection(node *invertibleTypeJoin, parentPlan if len(indexes) > 0 && !filter.IsComplex(parentPlan.selectNode.filter) { subInd := node.documentMapping.FirstIndexOfName(node.parentSide.relFieldDef.Value().Name) relatedField := mapper.Field{Name: node.parentSide.relFieldDef.Value().Name, Index: subInd} - fieldFilter := filter.UnwrapRelation(filter.CopyField( - parentPlan.selectNode.filter, - relatedField, - mapper.Field{Name: subFieldName, Index: subFieldInd}, - ), relatedField) + relevantFilter := filter.CopyField(parentPlan.selectNode.filter, relatedField, + mapper.Field{Name: subFieldName, Index: subFieldInd}) + fieldFilter := extractRelatedSubFilter(relevantFilter, node.parentSide.plan.DocumentMap(), relatedField) // At the moment we just take the first index, but later we want to run some kind of analysis to // determine which index is best to use. https://github.com/sourcenetwork/defradb/issues/2680 err := node.invertJoinDirectionWithIndex(fieldFilter, indexes[0]) @@ -327,6 +325,13 @@ func (p *Planner) tryOptimizeJoinDirection(node *invertibleTypeJoin, parentPlan return nil } +func extractRelatedSubFilter(f *mapper.Filter, docMap *core.DocumentMapping, relField mapper.Field) *mapper.Filter { + subInd := docMap.FirstIndexOfName(relField.Name) + relatedField := mapper.Field{Name: relField.Name, Index: subInd} + subFilter := filter.UnwrapRelation(f, relatedField) + return subFilter +} + // expandTypeJoin does a plan graph expansion and other optimizations on invertibleTypeJoin. func (p *Planner) expandTypeJoin(node *invertibleTypeJoin, parentPlan *selectTopNode) error { if parentPlan.selectNode.filter == nil { diff --git a/internal/planner/scan.go b/internal/planner/scan.go index bfca64b7bb..161a28f2f4 100644 --- a/internal/planner/scan.go +++ b/internal/planner/scan.go @@ -20,7 +20,6 @@ import ( "github.com/sourcenetwork/defradb/internal/db/fetcher" "github.com/sourcenetwork/defradb/internal/keys" "github.com/sourcenetwork/defradb/internal/lens" - "github.com/sourcenetwork/defradb/internal/planner/filter" "github.com/sourcenetwork/defradb/internal/planner/mapper" "github.com/sourcenetwork/defradb/internal/request/graphql/parser" ) @@ -51,6 +50,7 @@ type scanNode struct { filter *mapper.Filter slct *mapper.Select + index immutable.Option[client.IndexDescription] fetcher fetcher.Fetcher execInfo scanExecInfo @@ -67,6 +67,7 @@ func (n *scanNode) Init() error { n.p.identity, n.p.txn, n.p.acp, + n.index, n.col, n.fields, n.filter, @@ -152,46 +153,13 @@ func (n *scanNode) addField(field client.FieldDefinition) { } } -func (scan *scanNode) initFetcher( - cid immutable.Option[string], - index immutable.Option[client.IndexDescription], -) { +func (scan *scanNode) initFetcher(cid immutable.Option[string]) { var f fetcher.Fetcher if cid.HasValue() { f = new(fetcher.VersionedFetcher) } else { f = fetcher.NewDocumentFetcher() - if index.HasValue() { - fieldsToMove := make([]mapper.Field, 0, len(index.Value().Fields)) - fieldsToCopy := make([]mapper.Field, 0, len(index.Value().Fields)) - for _, field := range index.Value().Fields { - fieldName := field.Name - typeIndex := scan.documentMapping.FirstIndexOfName(fieldName) - indexField := mapper.Field{Index: typeIndex, Name: fieldName} - fd, _ := scan.col.Definition().Schema.GetFieldByName(fieldName) - // if the field is an array, we need to copy it instead of moving so that the - // top select node can do final filter check on the whole array of the document - // because indexes can not assert conditions like _any, _all, _none - // TODO: we don't have to do this for all json fields, only for those that filter - // on it's array fields. We should be able to optimize this. - // https://github.com/sourcenetwork/defradb/issues/3306 - if fd.Kind.IsArray() || fd.Kind == client.FieldKind_NILLABLE_JSON { - fieldsToCopy = append(fieldsToCopy, indexField) - } else { - fieldsToMove = append(fieldsToMove, indexField) - } - } - var indexFilter *mapper.Filter - scan.filter, indexFilter = filter.SplitByFields(scan.filter, fieldsToMove...) - for i := range fieldsToCopy { - indexFilter = filter.Merge(indexFilter, filter.CopyField(scan.filter, fieldsToCopy[i])) - } - if indexFilter != nil { - f = fetcher.NewIndexFetcher(f, index.Value(), indexFilter) - } - } - f = lens.NewFetcher(f, scan.p.db.LensRegistry()) } scan.fetcher = f diff --git a/internal/planner/select.go b/internal/planner/select.go index d3bcbb910d..8d41d1d251 100644 --- a/internal/planner/select.go +++ b/internal/planner/select.go @@ -11,6 +11,9 @@ package planner import ( + "slices" + "strings" + cid "github.com/ipfs/go-cid" "github.com/sourcenetwork/immutable" @@ -296,7 +299,8 @@ func (n *selectNode) initSource() ([]aggregateNode, error) { } if isScanNode { - origScan.initFetcher(n.selectReq.Cid, findIndexByFilteringField(origScan)) + origScan.index = findIndexByFilteringField(origScan) + origScan.initFetcher(n.selectReq.Cid) } return aggregates, nil @@ -308,17 +312,31 @@ func findIndexByFilteringField(scanNode *scanNode) immutable.Option[client.Index } colDesc := scanNode.col.Description() - for _, field := range scanNode.col.Schema().Fields { - if _, isFiltered := scanNode.filter.ExternalConditions[field.Name]; !isFiltered { - continue - } - indexes := colDesc.GetIndexesOnField(field.Name) - if len(indexes) > 0 { - // we return the first found index. We will optimize it later. - return immutable.Some(indexes[0]) + conditions := scanNode.filter.ExternalConditions + var indexCandidates []client.IndexDescription + filter.TraverseFields(conditions, func(path []string, val any) bool { + for _, field := range scanNode.col.Schema().Fields { + if field.Name != path[0] { + continue + } + indexes := colDesc.GetIndexesOnField(field.Name) + if len(indexes) > 0 { + indexCandidates = append(indexCandidates, indexes...) + return true + } } + return true + }) + if len(indexCandidates) == 0 { + return immutable.None[client.IndexDescription]() } - return immutable.None[client.IndexDescription]() + + slices.SortFunc(indexCandidates, func(a, b client.IndexDescription) int { + return strings.Compare(a.Name, b.Name) + }) + // we return the first found index. We will optimize it later. + // https://github.com/sourcenetwork/defradb/issues/2680 + return immutable.Some(indexCandidates[0]) } func findIndexByFieldName(col client.Collection, fieldName string) immutable.Option[client.IndexDescription] { diff --git a/internal/planner/type_join.go b/internal/planner/type_join.go index 5e7b83d237..5012d6f6a3 100644 --- a/internal/planner/type_join.go +++ b/internal/planner/type_join.go @@ -372,12 +372,16 @@ func (p *Planner) newInvertableTypeJoin( isParent: false, } - return invertibleTypeJoin{ + join := invertibleTypeJoin{ docMapper: docMapper{parent.documentMapping}, parentSide: parentSide, childSide: childSide, skipChild: skipChild, - }, nil + // we store child's own filter in case an index kicks in and replaces it with it's own filter + subFilter: getScanNode(childSide.plan).filter, + } + + return join, nil } type joinSide struct { @@ -437,11 +441,11 @@ func getForeignKey(node planNode, relFieldName string) string { return docIDStr } -// fetchDocWithID fetches a document with the given docID from the given planNode. -func fetchDocWithID(node planNode, docID string) (bool, error) { +// fetchDocWithIDAndItsSubDocs fetches a document with the given docID from the given planNode. +func fetchDocWithIDAndItsSubDocs(node planNode, docID string) (immutable.Option[core.Doc], error) { scan := getScanNode(node) if scan == nil { - return false, nil + return immutable.None[core.Doc](), nil } dsKey := base.MakeDataStoreKeyWithCollectionAndDocID(scan.col.Description(), docID) @@ -450,16 +454,16 @@ func fetchDocWithID(node planNode, docID string) (bool, error) { node.Prefixes(prefixes) if err := node.Init(); err != nil { - return false, NewErrSubTypeInit(err) + return immutable.None[core.Doc](), NewErrSubTypeInit(err) } hasValue, err := node.Next() if err != nil || !hasValue { - return false, err + return immutable.None[core.Doc](), err } - return true, nil + return immutable.Some(node.Value()), nil } type invertibleTypeJoin struct { @@ -470,6 +474,9 @@ type invertibleTypeJoin struct { parentSide joinSide childSide joinSide + // the filter of the subnode to store in case it's replaced by an index filter + subFilter *mapper.Filter + secondaryFetchLimit uint // docsToYield contains documents read and ready to be yielded by this node. @@ -514,22 +521,15 @@ type primaryObjectsRetriever struct { primarySide *joinSide secondarySide *joinSide + targetSecondaryDoc core.Doc + filter *mapper.Filter + primaryScan *scanNode resultPrimaryDocs []core.Doc resultSecondaryDoc core.Doc } -func newPrimaryObjectsRetriever( - primarySide, secondarySide *joinSide, -) primaryObjectsRetriever { - j := primaryObjectsRetriever{ - primarySide: primarySide, - secondarySide: secondarySide, - } - return j -} - func (r *primaryObjectsRetriever) retrievePrimaryDocsReferencingSecondaryDoc() error { relIDFieldDef, ok := r.primarySide.col.Definition().GetFieldByName( r.primarySide.relFieldDef.Value().Name + request.RelatedObjectID) @@ -547,7 +547,12 @@ func (r *primaryObjectsRetriever) retrievePrimaryDocsReferencingSecondaryDoc() e return err } - r.resultPrimaryDocs, r.resultSecondaryDoc = joinPrimaryDocs(primaryDocs, r.secondarySide, r.primarySide) + r.resultPrimaryDocs, r.resultSecondaryDoc = joinPrimaryDocs( + primaryDocs, + r.targetSecondaryDoc, + r.primarySide, + r.secondarySide, + ) return nil } @@ -585,13 +590,14 @@ func (r *primaryObjectsRetriever) collectDocs(numDocs int) ([]core.Doc, error) { func (r *primaryObjectsRetriever) retrievePrimaryDocs() ([]core.Doc, error) { r.primaryScan.addField(r.relIDFieldDef) - secondaryDoc := r.secondarySide.plan.Value() - addFilterOnIDField(r.primaryScan, r.primarySide.relIDFieldMapIndex.Value(), secondaryDoc.GetID()) + r.primaryScan.filter = addFilterOnIDField(r.filter, r.primarySide.relIDFieldMapIndex.Value(), + r.targetSecondaryDoc.GetID()) oldFetcher := r.primaryScan.fetcher + oldIndex := r.primaryScan.index - indexOnRelation := findIndexByFieldName(r.primaryScan.col, r.relIDFieldDef.Name) - r.primaryScan.initFetcher(immutable.None[string](), indexOnRelation) + r.primaryScan.index = findIndexByFieldName(r.primaryScan.col, r.relIDFieldDef.Name) + r.primaryScan.initFetcher(immutable.None[string]()) docs, err := r.collectDocs(0) if err != nil { @@ -604,6 +610,7 @@ func (r *primaryObjectsRetriever) retrievePrimaryDocs() ([]core.Doc, error) { } r.primaryScan.fetcher = oldFetcher + r.primaryScan.index = oldIndex return docs, nil } @@ -616,9 +623,11 @@ func docsToDocIDs(docs []core.Doc) []string { return docIDs } -func joinPrimaryDocs(primaryDocs []core.Doc, secondarySide, primarySide *joinSide) ([]core.Doc, core.Doc) { - secondaryDoc := secondarySide.plan.Value() - +func joinPrimaryDocs( + primaryDocs []core.Doc, + secondaryDoc core.Doc, + primarySide, secondarySide *joinSide, +) ([]core.Doc, core.Doc) { if secondarySide.relFieldMapIndex.HasValue() { if !secondarySide.relFieldDef.HasValue() || secondarySide.relFieldDef.Value().Kind.IsArray() { secondaryDoc.Fields[secondarySide.relFieldMapIndex.Value()] = primaryDocs @@ -650,8 +659,17 @@ func joinPrimaryDocs(primaryDocs []core.Doc, secondarySide, primarySide *joinSid return primaryDocs, secondaryDoc } -func (join *invertibleTypeJoin) fetchPrimaryDocsReferencingSecondaryDoc() ([]core.Doc, core.Doc, error) { - retriever := newPrimaryObjectsRetriever(join.getPrimarySide(), join.getSecondarySide()) +func fetchPrimaryDocsReferencingSecondaryDoc( + primarySide, secondarySide *joinSide, + secondaryDoc core.Doc, + filter *mapper.Filter, +) ([]core.Doc, core.Doc, error) { + retriever := primaryObjectsRetriever{ + primarySide: primarySide, + secondarySide: secondarySide, + targetSecondaryDoc: secondaryDoc, + filter: filter, + } err := retriever.retrievePrimaryDocsReferencingSecondaryDoc() return retriever.resultPrimaryDocs, retriever.resultSecondaryDoc, err } @@ -676,9 +694,10 @@ func (join *invertibleTypeJoin) Next() (bool, error) { } if firstSide.isPrimary() { - return join.nextJoinedSecondaryDoc() + return join.fetchRelatedSecondaryDocWithChildren(firstSide.plan.Value()) } else { - primaryDocs, secondaryDoc, err := join.fetchPrimaryDocsReferencingSecondaryDoc() + primaryDocs, secondaryDoc, err := fetchPrimaryDocsReferencingSecondaryDoc( + join.getPrimarySide(), join.getSecondarySide(), firstSide.plan.Value(), join.subFilter) if err != nil { return false, err } @@ -699,7 +718,7 @@ func (join *invertibleTypeJoin) Next() (bool, error) { return true, nil } -func (join *invertibleTypeJoin) nextJoinedSecondaryDoc() (bool, error) { +func (join *invertibleTypeJoin) fetchRelatedSecondaryDocWithChildren(primaryDoc core.Doc) (bool, error) { firstSide := join.getFirstSide() secondSide := join.getSecondSide() @@ -712,7 +731,9 @@ func (join *invertibleTypeJoin) nextJoinedSecondaryDoc() (bool, error) { return join.Next() } - if !firstSide.isParent { + if secondSide.isParent { + // child primary docs reference the same secondary parent doc. So if we already encountered + // the secondary parent doc, we continue to the next primary doc. for i := range join.encounteredDocIDs { if join.encounteredDocIDs[i] == secondaryDocID { return join.Next() @@ -721,12 +742,13 @@ func (join *invertibleTypeJoin) nextJoinedSecondaryDoc() (bool, error) { join.encounteredDocIDs = append(join.encounteredDocIDs, secondaryDocID) } - hasDoc, err := fetchDocWithID(secondSide.plan, secondaryDocID) + secondaryDocOpt, err := fetchDocWithIDAndItsSubDocs(secondSide.plan, secondaryDocID) + if err != nil { return false, err } - if !hasDoc { + if !secondaryDocOpt.HasValue() { if firstSide.isParent { join.docsToYield = append(join.docsToYield, firstSide.plan.Value()) return true, nil @@ -734,15 +756,18 @@ func (join *invertibleTypeJoin) nextJoinedSecondaryDoc() (bool, error) { return join.Next() } + secondaryDoc := secondaryDocOpt.Value() + if join.parentSide.relFieldDef.Value().Kind.IsArray() { var primaryDocs []core.Doc - var secondaryDoc core.Doc // if child is not requested as part of the response, we just add the existing one (fetched by the secondary index // on a filtered value) so that top select node that runs the filter again can yield it. if join.skipChild { - primaryDocs, secondaryDoc = joinPrimaryDocs([]core.Doc{firstSide.plan.Value()}, secondSide, firstSide) + primaryDocs, secondaryDoc = joinPrimaryDocs( + []core.Doc{firstSide.plan.Value()}, secondaryDoc, join.getPrimarySide(), join.getSecondSide()) } else { - primaryDocs, secondaryDoc, err = join.fetchPrimaryDocsReferencingSecondaryDoc() + primaryDocs, secondaryDoc, err = fetchPrimaryDocsReferencingSecondaryDoc( + join.getPrimarySide(), join.getSecondarySide(), secondaryDoc, join.subFilter) if err != nil { return false, err } @@ -751,8 +776,16 @@ func (join *invertibleTypeJoin) nextJoinedSecondaryDoc() (bool, error) { join.docsToYield = append(join.docsToYield, secondaryDoc) } else { - parentDoc := join.parentSide.plan.Value() - parentDoc.Fields[join.parentSide.relFieldMapIndex.Value()] = join.childSide.plan.Value() + var parentDoc core.Doc + var childDoc core.Doc + if join.getPrimarySide().isParent { + parentDoc = primaryDoc + childDoc = secondaryDoc + } else { + parentDoc = secondaryDoc + childDoc = primaryDoc + } + parentDoc.Fields[join.parentSide.relFieldMapIndex.Value()] = childDoc join.docsToYield = append(join.docsToYield, parentDoc) } return true, nil @@ -769,11 +802,13 @@ func (join *invertibleTypeJoin) invertJoinDirectionWithIndex( fieldFilter *mapper.Filter, index client.IndexDescription, ) error { - p := join.childSide.plan - s := getScanNode(p) - s.tryAddFieldWithName(join.childSide.relFieldDef.Value().Name + request.RelatedObjectID) - s.filter = fieldFilter - s.initFetcher(immutable.Option[string]{}, immutable.Some(index)) + childScan := getScanNode(join.childSide.plan) + childScan.tryAddFieldWithName(join.childSide.relFieldDef.Value().Name + request.RelatedObjectID) + // replace child's filter with the filter that utilizes the index + // the original child's filter is stored in join.subFilter + childScan.filter = fieldFilter + childScan.index = immutable.Some(index) + childScan.initFetcher(immutable.Option[string]{}) join.childSide.isFirst = join.parentSide.isFirst join.parentSide.isFirst = !join.parentSide.isFirst @@ -781,24 +816,21 @@ func (join *invertibleTypeJoin) invertJoinDirectionWithIndex( return nil } -func addFilterOnIDField(scan *scanNode, propIndex int, val any) { - if scan == nil { - return - } - - if scan.filter == nil { - scan.filter = mapper.NewFilter() +func addFilterOnIDField(f *mapper.Filter, propIndex int, docID string) *mapper.Filter { + if f == nil { + f = mapper.NewFilter() } propertyIndex := &mapper.PropertyIndex{Index: propIndex} filterConditions := map[connor.FilterKey]any{ propertyIndex: map[connor.FilterKey]any{ - mapper.FilterEqOp: val, + mapper.FilterEqOp: docID, }, } - filter.RemoveField(scan.filter, mapper.Field{Index: propIndex}) - scan.filter.Conditions = filter.MergeConditions(scan.filter.Conditions, filterConditions) + filter.RemoveField(f, mapper.Field{Index: propIndex}) + f.Conditions = filter.MergeConditions(f.Conditions, filterConditions) + return f } func getScanNode(plan planNode) *scanNode { diff --git a/tests/integration/explain/execute/create_test.go b/tests/integration/explain/execute/create_test.go index 29816b47c6..42ce15e334 100644 --- a/tests/integration/explain/execute/create_test.go +++ b/tests/integration/explain/execute/create_test.go @@ -48,7 +48,7 @@ func TestExecuteExplainMutationRequestWithCreate(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(1), - "fieldFetches": uint64(1), + "fieldFetches": uint64(3), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/delete_test.go b/tests/integration/explain/execute/delete_test.go index 6e43b3dbbd..8b90fe83f1 100644 --- a/tests/integration/explain/execute/delete_test.go +++ b/tests/integration/explain/execute/delete_test.go @@ -51,7 +51,7 @@ func TestExecuteExplainMutationRequestWithDeleteUsingID(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(1), - "fieldFetches": uint64(1), + "fieldFetches": uint64(2), "indexFetches": uint64(0), }, }, @@ -103,7 +103,7 @@ func TestExecuteExplainMutationRequestWithDeleteUsingFilter(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/top_level_test.go b/tests/integration/explain/execute/top_level_test.go index 784d7fe8a9..8d64ad2cbc 100644 --- a/tests/integration/explain/execute/top_level_test.go +++ b/tests/integration/explain/execute/top_level_test.go @@ -70,7 +70,7 @@ func TestExecuteExplainTopLevelAverageRequest(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(4), "indexFetches": uint64(0), }, }, @@ -233,7 +233,7 @@ func TestExecuteExplainTopLevelSumRequest(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(4), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/type_join_test.go b/tests/integration/explain/execute/type_join_test.go index 511c3498a9..be510237f8 100644 --- a/tests/integration/explain/execute/type_join_test.go +++ b/tests/integration/explain/execute/type_join_test.go @@ -54,13 +54,13 @@ func TestExecuteExplainRequestWithAOneToOneJoin(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(6), "indexFetches": uint64(0), }, }, @@ -120,13 +120,13 @@ func TestExecuteExplainWithMultipleOneToOneJoins(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(6), "indexFetches": uint64(0), }, }, @@ -137,13 +137,13 @@ func TestExecuteExplainWithMultipleOneToOneJoins(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(4), + "fieldFetches": uint64(6), "indexFetches": uint64(0), }, }, @@ -203,13 +203,13 @@ func TestExecuteExplainWithTwoLevelDeepNestedJoins(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(4), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(4), + "fieldFetches": uint64(6), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_average_test.go b/tests/integration/explain/execute/with_average_test.go index fdadf6fcaa..6bfe343a3c 100644 --- a/tests/integration/explain/execute/with_average_test.go +++ b/tests/integration/explain/execute/with_average_test.go @@ -58,7 +58,7 @@ func TestExecuteExplainAverageRequestOnArrayField(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(4), "docFetches": uint64(3), - "fieldFetches": uint64(5), + "fieldFetches": uint64(11), "indexFetches": uint64(0), }, }, @@ -120,13 +120,13 @@ func TestExplainExplainAverageRequestOnJoinedField(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(22), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_count_test.go b/tests/integration/explain/execute/with_count_test.go index 190a685592..cb623da565 100644 --- a/tests/integration/explain/execute/with_count_test.go +++ b/tests/integration/explain/execute/with_count_test.go @@ -56,13 +56,13 @@ func TestExecuteExplainRequestWithCountOnOneToManyRelation(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(6), + "fieldFetches": uint64(22), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_limit_test.go b/tests/integration/explain/execute/with_limit_test.go index ff2ca0b437..d4f1e5a846 100644 --- a/tests/integration/explain/execute/with_limit_test.go +++ b/tests/integration/explain/execute/with_limit_test.go @@ -53,7 +53,7 @@ func TestExecuteExplainRequestWithBothLimitAndOffsetOnParent(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(7), "indexFetches": uint64(0), }, }, @@ -111,13 +111,13 @@ func TestExecuteExplainRequestWithBothLimitAndOffsetOnParentAndLimitOnChild(t *t "scanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(2), "docFetches": uint64(3), - "fieldFetches": uint64(6), + "fieldFetches": uint64(9), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_max_test.go b/tests/integration/explain/execute/with_max_test.go index 6fa390be7c..900d526c5b 100644 --- a/tests/integration/explain/execute/with_max_test.go +++ b/tests/integration/explain/execute/with_max_test.go @@ -54,7 +54,7 @@ func TestExecuteExplainRequest_WithMaxOfInlineArrayField_Succeeds(t *testing.T) "scanNode": dataMap{ "iterations": uint64(4), "docFetches": uint64(3), - "fieldFetches": uint64(5), + "fieldFetches": uint64(11), "indexFetches": uint64(0), }, }, @@ -113,13 +113,13 @@ func TestExecuteExplainRequest_MaxOfRelatedOneToManyField_Succeeds(t *testing.T) "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(18), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_min_test.go b/tests/integration/explain/execute/with_min_test.go index 4626071d32..cad99db706 100644 --- a/tests/integration/explain/execute/with_min_test.go +++ b/tests/integration/explain/execute/with_min_test.go @@ -54,7 +54,7 @@ func TestExecuteExplainRequest_WithMinOfInlineArrayField_Succeeds(t *testing.T) "scanNode": dataMap{ "iterations": uint64(4), "docFetches": uint64(3), - "fieldFetches": uint64(5), + "fieldFetches": uint64(11), "indexFetches": uint64(0), }, }, @@ -113,13 +113,13 @@ func TestExecuteExplainRequest_MinOfRelatedOneToManyField_Succeeds(t *testing.T) "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(18), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_order_test.go b/tests/integration/explain/execute/with_order_test.go index bbb2312c38..0ea9530445 100644 --- a/tests/integration/explain/execute/with_order_test.go +++ b/tests/integration/explain/execute/with_order_test.go @@ -53,7 +53,7 @@ func TestExecuteExplainRequestWithOrderFieldOnParent(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(4), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, }, @@ -195,13 +195,13 @@ func TestExecuteExplainRequestWithOrderFieldOnChild(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(18), "indexFetches": uint64(0), }, }, @@ -260,13 +260,13 @@ func TestExecuteExplainRequestWithOrderFieldOnBothParentAndChild(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(4), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(18), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/explain/execute/with_sum_test.go b/tests/integration/explain/execute/with_sum_test.go index c7d614bf2f..736825c0da 100644 --- a/tests/integration/explain/execute/with_sum_test.go +++ b/tests/integration/explain/execute/with_sum_test.go @@ -54,7 +54,7 @@ func TestExecuteExplainRequestWithSumOfInlineArrayField(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(4), "docFetches": uint64(3), - "fieldFetches": uint64(5), + "fieldFetches": uint64(11), "indexFetches": uint64(0), }, }, @@ -113,13 +113,13 @@ func TestExecuteExplainRequestSumOfRelatedOneToManyField(t *testing.T) { "scanNode": dataMap{ "iterations": uint64(3), "docFetches": uint64(2), - "fieldFetches": uint64(2), + "fieldFetches": uint64(8), "indexFetches": uint64(0), }, "subTypeScanNode": dataMap{ "iterations": uint64(5), "docFetches": uint64(6), - "fieldFetches": uint64(12), + "fieldFetches": uint64(18), "indexFetches": uint64(0), }, }, diff --git a/tests/integration/index/array_test.go b/tests/integration/index/array_test.go index 096ecb87e1..f1ee1b56ef 100644 --- a/tests/integration/index/array_test.go +++ b/tests/integration/index/array_test.go @@ -122,7 +122,7 @@ func TestArrayIndex_WithFilterOnIndexedArrayUsingAll_ShouldUseIndex(t *testing.T testUtils.ExecuteTestCase(t, test) } -func TestArrayIndex_WithFilterOnIndexedArrayUsingNone_ShouldUseIndex(t *testing.T) { +func TestArrayIndex_WithFilterOnIndexedArrayUsingNone_ShouldNotUseIndex(t *testing.T) { req := `query { User(filter: {numbers: {_none: {_ge: 33}}}) { name @@ -166,8 +166,9 @@ func TestArrayIndex_WithFilterOnIndexedArrayUsingNone_ShouldUseIndex(t *testing. }, }, testUtils.Request{ - Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithIndexFetches(9), + Request: makeExplainQuery(req), + // index is not used for _none operator as it might be even less optimal than full scan + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(0), }, }, } diff --git a/tests/integration/index/json_composite_test.go b/tests/integration/index/json_composite_test.go index 187ed99334..9a0bfc867b 100644 --- a/tests/integration/index/json_composite_test.go +++ b/tests/integration/index/json_composite_test.go @@ -222,7 +222,7 @@ func TestJSONCompositeIndex_JSONWithScalarWithOtherFilters_ShouldFetchUsingIndex req: `query { User(filter: { age: {_le: 35}, - custom: {val: {_gt: 4}} + custom: {val: {_gt: 4}} }) { name } @@ -240,7 +240,7 @@ func TestJSONCompositeIndex_JSONWithScalarWithOtherFilters_ShouldFetchUsingIndex req: `query { User(filter: { age: {_lt: 100}, - custom: {val: {_eq: null}} + custom: {val: {_eq: null}} }) { name } @@ -267,7 +267,7 @@ func TestJSONCompositeIndex_JSONWithScalarWithOtherFilters_ShouldFetchUsingIndex {"name": "Chris"}, }, }, - indexFetches: 0, + indexFetches: 8, }, } diff --git a/tests/integration/index/json_test.go b/tests/integration/index/json_test.go index d154248501..749192a105 100644 --- a/tests/integration/index/json_test.go +++ b/tests/integration/index/json_test.go @@ -419,8 +419,10 @@ func TestJSONIndex_WithNeFilterOnNumberField_ShouldUseIndex(t *testing.T) { Request: req, Results: map[string]any{ "User": []map[string]any{ + {"name": "Bruno"}, {"name": "John"}, {"name": "Andy"}, + {"name": "Keenan"}, }, }, }, @@ -803,6 +805,8 @@ func TestJSONIndex_WithNeFilterOnBoolField_ShouldUseIndex(t *testing.T) { Request: req, Results: map[string]any{ "User": []map[string]any{ + {"name": "Bruno"}, + {"name": "Keenan"}, {"name": "Islam"}, {"name": "John"}, }, @@ -1301,12 +1305,239 @@ func TestJSONIndex_WithCompoundFilterCondition_ShouldUseIndex(t *testing.T) { }, }, testUtils.Request{ - Request: makeExplainQuery(req), - // TODO: this test doesn't utilize indexes. https://github.com/sourcenetwork/defradb/issues/3299 - Asserter: testUtils.NewExplainAsserter().WithIndexFetches(0), + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestJSONIndex_WithNeFilterAgainstNumberField_ShouldFetchNullValues(t *testing.T) { + req := `query { + User(filter: {custom: {age: {_ne: 48}}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "age": 48, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "age": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "age": 42, + }, + }, + }, + testUtils.Request{ + Request: req, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestJSONIndex_WithNeFilterAgainstStringField_ShouldFetchNullValues(t *testing.T) { + req := `query { + User(filter: {custom: {city: {_ne: "Istanbul"}}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "city": "Istanbul", + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "city": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "city": "Lucerne", + }, + }, + }, + testUtils.Request{ + Request: req, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } + testUtils.ExecuteTestCase(t, test) +} +func TestJSONIndex_WithNeFilterAgainstBoolField_ShouldFetchNullValues(t *testing.T) { + req := `query { + User(filter: {custom: {verified: {_ne: true}}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "verified": true, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "verified": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "verified": false, + }, + }, + }, + testUtils.Request{ + Request: req, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestJSONIndex_WithNeFilterAgainstNullField_ShouldFetchNonNullValues(t *testing.T) { + req := `query { + User(filter: {custom: {age: {_ne: null}}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "age": 48, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "age": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "age": 42, + }, + }, + }, + testUtils.Request{ + Request: req, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "John"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } testUtils.ExecuteTestCase(t, test) } diff --git a/tests/integration/index/query_with_composite_index_only_filter_test.go b/tests/integration/index/query_with_composite_index_only_filter_test.go index 63e704cc78..c2dfcd898c 100644 --- a/tests/integration/index/query_with_composite_index_only_filter_test.go +++ b/tests/integration/index/query_with_composite_index_only_filter_test.go @@ -59,7 +59,7 @@ func TestQueryWithCompositeIndex_WithEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, testUtils.Request{ Request: req2, @@ -71,7 +71,7 @@ func TestQueryWithCompositeIndex_WithEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, testUtils.Request{ Request: req3, @@ -115,7 +115,7 @@ func TestQueryWithCompositeIndex_WithGreaterThanFilterOnFirstField_ShouldFetch(t }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -153,7 +153,7 @@ func TestQueryWithCompositeIndex_WithGreaterThanFilterOnSecondField_ShouldFetch( }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -192,7 +192,7 @@ func TestQueryWithCompositeIndex_WithGreaterOrEqualFilterOnFirstField_ShouldFetc }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -231,7 +231,7 @@ func TestQueryWithCompositeIndex_WithGreaterOrEqualFilterOnSecondField_ShouldFet }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -269,7 +269,7 @@ func TestQueryWithCompositeIndex_WithLessThanFilterOnFirstField_ShouldFetch(t *t }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -307,7 +307,7 @@ func TestQueryWithCompositeIndex_WithLessThanFilterOnSecondField_ShouldFetch(t * }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -346,7 +346,7 @@ func TestQueryWithCompositeIndex_WithLessOrEqualFilterOnFirstField_ShouldFetch(t }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -385,7 +385,7 @@ func TestQueryWithCompositeIndex_WithLessOrEqualFilterOnSecondField_ShouldFetch( }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -430,7 +430,7 @@ func TestQueryWithCompositeIndex_WithNotEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -469,7 +469,7 @@ func TestQueryWithCompositeIndex_WithInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } @@ -509,7 +509,7 @@ func TestQueryWithCompositeIndex_WithNotInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -576,7 +576,7 @@ func TestQueryWithCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req2, @@ -588,7 +588,7 @@ func TestQueryWithCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req3, @@ -600,7 +600,7 @@ func TestQueryWithCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req3), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req4, @@ -612,7 +612,7 @@ func TestQueryWithCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req4), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req5, @@ -624,7 +624,7 @@ func TestQueryWithCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req5), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req6, @@ -676,7 +676,7 @@ func TestQueryWithCompositeIndex_WithNotLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -705,7 +705,7 @@ func TestQueryWithCompositeIndex_IfFirstFieldIsNotInFilter_ShouldNotUseIndex(t * name } }`, - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(20).WithIndexFetches(0), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(0), }, }, } @@ -968,7 +968,7 @@ func TestQueryWithCompositeIndex_IfConsecutiveEqOps_ShouldUseAllToOptimizeQuery( }, testUtils.Request{ Request: makeExplainQuery(reqWithName), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(4).WithIndexFetches(4), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(4), }, testUtils.Request{ Request: reqWithNameAge, @@ -982,7 +982,7 @@ func TestQueryWithCompositeIndex_IfConsecutiveEqOps_ShouldUseAllToOptimizeQuery( }, testUtils.Request{ Request: makeExplainQuery(reqWithNameAge), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(3).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, testUtils.Request{ Request: reqWithNameAgeNumChildren, @@ -995,7 +995,7 @@ func TestQueryWithCompositeIndex_IfConsecutiveEqOps_ShouldUseAllToOptimizeQuery( }, testUtils.Request{ Request: makeExplainQuery(reqWithNameAgeNumChildren), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } diff --git a/tests/integration/index/query_with_index_combined_filter_test.go b/tests/integration/index/query_with_index_combined_filter_test.go index 0a6251db46..9d0572d2df 100644 --- a/tests/integration/index/query_with_index_combined_filter_test.go +++ b/tests/integration/index/query_with_index_combined_filter_test.go @@ -48,7 +48,7 @@ func TestQueryWithIndex_IfIndexFilterWithRegular_ShouldFilter(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(3).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } @@ -90,7 +90,7 @@ func TestQueryWithIndex_IfMultipleIndexFiltersWithRegular_ShouldFilter(t *testin }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(12), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -132,7 +132,7 @@ func TestQueryWithIndex_IfMultipleIndexFiltersWithRegularCaseInsensitive_ShouldF }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(6), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } diff --git a/tests/integration/index/query_with_index_on_datetime_test.go b/tests/integration/index/query_with_index_on_datetime_test.go index 6efb8d5644..8633c626d6 100644 --- a/tests/integration/index/query_with_index_on_datetime_test.go +++ b/tests/integration/index/query_with_index_on_datetime_test.go @@ -53,7 +53,7 @@ func TestQueryWithIndex_WithEqFilterOnDateTimeField_ShouldIndex(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } diff --git a/tests/integration/index/query_with_index_only_filter_test.go b/tests/integration/index/query_with_index_only_filter_test.go index f0aab40546..8411ebaa2a 100644 --- a/tests/integration/index/query_with_index_only_filter_test.go +++ b/tests/integration/index/query_with_index_only_filter_test.go @@ -49,7 +49,7 @@ func TestQueryWithIndex_WithNonIndexedFields_ShouldFetchAllOfThem(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } @@ -85,7 +85,7 @@ func TestQueryWithIndex_WithEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } @@ -130,7 +130,7 @@ func TestQueryWithIndex_IfSeveralDocsWithEqFilter_ShouldFetchAll(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -167,7 +167,7 @@ func TestQueryWithIndex_WithGreaterThanFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -205,7 +205,7 @@ func TestQueryWithIndex_WithGreaterOrEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -242,7 +242,7 @@ func TestQueryWithIndex_WithLessThanFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -280,7 +280,7 @@ func TestQueryWithIndex_WithLessOrEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -325,7 +325,7 @@ func TestQueryWithIndex_WithNotEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -363,7 +363,7 @@ func TestQueryWithIndex_WithInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -463,7 +463,7 @@ func TestQueryWithIndex_IfSeveralDocsWithInFilter_ShouldFetchAll(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -503,7 +503,7 @@ func TestQueryWithIndex_WithNotInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(4).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -566,7 +566,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req2, @@ -579,7 +579,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req3, @@ -592,7 +592,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req3), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req4, @@ -604,7 +604,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req4), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req5, @@ -617,7 +617,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req5), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req6, @@ -627,7 +627,7 @@ func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req6), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -670,7 +670,7 @@ func TestQueryWithIndex_WithNotLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -804,3 +804,348 @@ func TestQueryWithIndex_WithFilterOn2Relations_ShouldFilter(t *testing.T) { testUtils.ExecuteTestCase(t, test) } + +func TestQueryWithIndex_WithNeFilterAgainstIntField_ShouldFetchNilValues(t *testing.T) { + req1 := `query { + User(filter: {age: {_ne: 48}}) { + name + } + }` + req2 := `query { + User(filter: {age: {_ne: null}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + age: Int @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "age": 48, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "age": nil, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "age": 42, + }, + }, + testUtils.Request{ + Request: req1, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "John"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNeFilterAgainstFloatField_ShouldFetchNilValues(t *testing.T) { + req1 := `query { + User(filter: {rating: {_ne: 4.5}}) { + name + } + }` + req2 := `query { + User(filter: {rating: {_ne: null}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + rating: Float @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "rating": 4.5, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "rating": nil, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "rating": 4.2, + }, + }, + testUtils.Request{ + Request: req1, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "John"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNeFilterAgainstStringField_ShouldFetchNilValues(t *testing.T) { + req1 := `query { + User(filter: {city: {_ne: "Istanbul"}}) { + name + } + }` + req2 := `query { + User(filter: {city: {_ne: null}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + city: String @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "city": "Istanbul", + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "city": nil, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "city": "Lucerne", + }, + }, + testUtils.Request{ + Request: req1, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "John"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNeFilterAgainstDateTimeField_ShouldFetchNilValues(t *testing.T) { + req1 := `query { + User(filter: {birthdate: {_ne: "2020-01-01T00:00:00Z"}}) { + name + } + }` + req2 := `query { + User(filter: {birthdate: {_ne: null}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + birthdate: DateTime @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "birthdate": "2020-01-01T00:00:00Z", + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "birthdate": nil, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "birthdate": "2024-01-01T00:00:00Z", + }, + }, + testUtils.Request{ + Request: req1, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "John"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNeFilterAgainstBooleanField_ShouldFetchNilValues(t *testing.T) { + req1 := `query { + User(filter: {verified: {_ne: true}}) { + name + } + }` + req2 := `query { + User(filter: {verified: {_ne: null}}) { + name + } + }` + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + verified: Boolean @index + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "verified": true, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "verified": nil, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "verified": false, + }, + }, + testUtils.Request{ + Request: req1, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Andy"}, + {"name": "Shahzad"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "John"}, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/index/query_with_relation_filter_test.go b/tests/integration/index/query_with_relation_filter_test.go index fa7fb6c443..ee1dd21e41 100644 --- a/tests/integration/index/query_with_relation_filter_test.go +++ b/tests/integration/index/query_with_relation_filter_test.go @@ -16,74 +16,6 @@ import ( testUtils "github.com/sourcenetwork/defradb/tests/integration" ) -func TestQueryWithIndexOnOneToManyRelation_IfFilterOnIndexedRelation_ShouldFilter2(t *testing.T) { - // 3 users have a MacBook Pro: Islam, Shahzad, Keenan - req1 := `query { - User(filter: { - devices: {model: {_eq: "MacBook Pro"}} - }) { - name - } - }` - // 1 user has an iPhone 10: Addo - req2 := `query { - User(filter: { - devices: {model: {_eq: "iPhone 10"}} - }) { - name - } - }` - test := testUtils.TestCase{ - Description: "Filter on indexed relation field in 1-N relation", - Actions: []any{ - testUtils.SchemaUpdate{ - Schema: ` - type User { - name: String - age: Int - devices: [Device] - } - - type Device { - model: String @index - owner: User - }`, - }, - testUtils.CreatePredefinedDocs{ - Docs: getUserDocs(), - }, - testUtils.Request{ - Request: req1, - Results: map[string]any{ - "User": []map[string]any{ - {"name": "Shahzad"}, - {"name": "Islam"}, - {"name": "Keenan"}, - }, - }, - }, - testUtils.Request{ - Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(6).WithIndexFetches(3), - }, - testUtils.Request{ - Request: req2, - Results: map[string]any{ - "User": []map[string]any{ - {"name": "Addo"}, - }, - }, - }, - testUtils.Request{ - Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(1), - }, - }, - } - - testUtils.ExecuteTestCase(t, test) -} - func TestQueryWithIndexOnOneToManyRelation_IfFilterOnIndexedRelation_ShouldFilter(t *testing.T) { // 3 users have a MacBook Pro: Islam, Shahzad, Keenan req1 := `query { @@ -132,7 +64,7 @@ func TestQueryWithIndexOnOneToManyRelation_IfFilterOnIndexedRelation_ShouldFilte }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(6).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, testUtils.Request{ Request: req2, @@ -144,7 +76,7 @@ func TestQueryWithIndexOnOneToManyRelation_IfFilterOnIndexedRelation_ShouldFilte }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } @@ -198,7 +130,7 @@ func TestQueryWithIndexOnOneToOnesSecondaryRelation_IfFilterOnIndexedRelation_Sh }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, testUtils.Request{ Request: req2, @@ -212,7 +144,7 @@ func TestQueryWithIndexOnOneToOnesSecondaryRelation_IfFilterOnIndexedRelation_Sh }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(6).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } @@ -269,8 +201,7 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedFieldOfRelatio Request: makeExplainQuery(req1), // we make 2 index fetches: 1. to get the only address with city == "London" // and 2. to get the corresponding user - // then 1 field fetch to get the name of the user - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, testUtils.Request{ Request: req2, @@ -286,8 +217,7 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedFieldOfRelatio Request: makeExplainQuery(req2), // we make 3 index fetches to get the 3 address with city == "Montreal" // and 3 more index fetches to get the corresponding users - // then 3 field fetches to get the name of each user - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(3).WithIndexFetches(6), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(6), }, }, } @@ -343,9 +273,10 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedFieldOfRelatio testUtils.Request{ Request: makeExplainQuery(req1), // we make 1 index fetch to get the only address with city == "London" + // we fetch 2 fields for Address doc: "city" and "street" // then we scan all 10 users to find one with matching "address_id" - // after this we fetch the name of the user - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(20).WithIndexFetches(1), + // for each of User docs we fetch 3 fields: "name", "age" and "address_id" + Asserter: testUtils.NewExplainAsserter().WithFieldFetches(32).WithIndexFetches(1), }, testUtils.Request{ Request: req2, @@ -362,7 +293,7 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedFieldOfRelatio // we make 3 index fetch to get the 3 address with city == "Montreal" // then we scan all 10 users to find one with matching "address_id" for each address // after this we fetch the name of each user - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(60).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } @@ -409,7 +340,7 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedRelationWhileI }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -417,7 +348,7 @@ func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedRelationWhileI testUtils.ExecuteTestCase(t, test) } -func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedRelation_ShouldFilter(t *testing.T) { +func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedPrimaryDoc_ShouldFilter(t *testing.T) { test := testUtils.TestCase{ Description: "Filter on indexed relation field in 1-N relations", Actions: []any{ @@ -512,6 +443,96 @@ func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedRelation_ShouldFilter(t *tes testUtils.ExecuteTestCase(t, test) } +func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedPrimaryDocAndSubFilter_ShouldFilter(t *testing.T) { + test := testUtils.TestCase{ + Description: "Filter on indexed relation field in 1-N relations", + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + devices: [Device] + } + + type Device { + model: String @index + manufacturer: String + owner: User + } + `, + }, + testUtils.CreateDoc{ + CollectionID: 0, + Doc: `{ + "name": "Chris" + }`, + }, + testUtils.CreateDoc{ + CollectionID: 0, + Doc: `{ + "name": "Addo" + }`, + }, + testUtils.CreateDoc{ + CollectionID: 1, + DocMap: map[string]any{ + "model": "Walkman", + "manufacturer": "Sony", + "owner": testUtils.NewDocIndex(0, 0), + }, + }, + testUtils.CreateDoc{ + CollectionID: 1, + DocMap: map[string]any{ + "model": "Walkman", + "manufacturer": "The Proclaimers", + "owner": testUtils.NewDocIndex(0, 0), + }, + }, + testUtils.CreateDoc{ + CollectionID: 1, + DocMap: map[string]any{ + "model": "Running Man", + "manufacturer": "Braveworld Productions", + "owner": testUtils.NewDocIndex(0, 0), + }, + }, + testUtils.Request{ + Request: `query { + User(filter: { + devices: {model: {_eq: "Walkman"}} + }) { + name + devices(filter: {manufacturer: {_ne: "Sony"}}) { + model + manufacturer + } + } + }`, + Results: map[string]any{ + "User": []map[string]any{ + { + "name": "Chris", + "devices": []map[string]any{ + { + "model": "Walkman", + "manufacturer": "The Proclaimers", + }, + { + "model": "Running Man", + "manufacturer": "Braveworld Productions", + }, + }, + }, + }, + }, + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedRelation_ShouldFilterWithExplain(t *testing.T) { req := `query { User(filter: { @@ -597,7 +618,7 @@ func TestQueryWithIndexOnOneToMany_IfFilterOnIndexedRelation_ShouldFilterWithExp }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(14).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -651,7 +672,7 @@ func TestQueryWithIndexOnOneToOne_IfFilterOnIndexedRelation_ShouldFilter(t *test }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } @@ -720,8 +741,7 @@ func TestQueryWithIndexOnManyToOne_IfFilterOnIndexedField_ShouldFilterWithExplai testUtils.Request{ Request: makeExplainQuery(req), // we make 3 index fetches to get all 3 devices with year 2021 - // and 9 field fetches: for every device we fetch additionally "model", "owner_id" and owner's "name" - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(9).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, }, } @@ -773,8 +793,7 @@ func TestQueryWithIndexOnManyToOne_IfFilterOnIndexedRelation_ShouldFilterWithExp Request: makeExplainQuery(req), // we make 1 index fetch to get the owner by it's name // and 3 index fetches to get all 3 devices of the owner - // and 3 field fetches to get 1 'model' field for every fetched device. - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(3).WithIndexFetches(4), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(4), }, }, } @@ -855,9 +874,7 @@ func TestQueryWithIndexOnOneToMany_IfIndexedRelationIsNil_NeNilFilterShouldUseIn testUtils.Request{ Request: makeExplainQuery(req), // we make 4 index fetches to find 2 devices with owner_id != null - // and 2 field fetches to get 1 'model' field for every fetched device - // plus 2 more field fetches to get related User docs - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(4).WithIndexFetches(4), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(4), }, }, } @@ -938,8 +955,7 @@ func TestQueryWithIndexOnOneToMany_IfIndexedRelationIsNil_EqNilFilterShouldUseIn testUtils.Request{ Request: makeExplainQuery(req), // we make 2 index fetches to get all 2 devices with owner_id == null - // and 2 field fetches to get 1 'model' field for every fetched device. - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } diff --git a/tests/integration/index/query_with_unique_composite_index_filter_test.go b/tests/integration/index/query_with_unique_composite_index_filter_test.go index 9928cb684b..3d041a0d66 100644 --- a/tests/integration/index/query_with_unique_composite_index_filter_test.go +++ b/tests/integration/index/query_with_unique_composite_index_filter_test.go @@ -77,7 +77,7 @@ func TestQueryWithUniqueCompositeIndex_WithEqualFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(3), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(3), }, testUtils.Request{ Request: req2, @@ -89,7 +89,7 @@ func TestQueryWithUniqueCompositeIndex_WithEqualFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, testUtils.Request{ Request: req3, @@ -133,7 +133,7 @@ func TestQueryWithUniqueCompositeIndex_WithGreaterThanFilterOnFirstField_ShouldF }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -171,7 +171,7 @@ func TestQueryWithUniqueCompositeIndex_WithGreaterThanFilterOnSecondField_Should }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -210,7 +210,7 @@ func TestQueryWithUniqueCompositeIndex_WithGreaterOrEqualFilterOnFirstField_Shou }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -249,7 +249,7 @@ func TestQueryWithUniqueCompositeIndex_WithGreaterOrEqualFilterOnSecondField_Sho }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -287,7 +287,7 @@ func TestQueryWithUniqueCompositeIndex_WithLessThanFilterOnFirstField_ShouldFetc }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -325,7 +325,7 @@ func TestQueryWithUniqueCompositeIndex_WithLessThanFilterOnSecondField_ShouldFet }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -364,7 +364,7 @@ func TestQueryWithUniqueCompositeIndex_WithLessOrEqualFilterOnFirstField_ShouldF }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -403,7 +403,7 @@ func TestQueryWithUniqueCompositeIndex_WithLessOrEqualFilterOnSecondField_Should }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -448,7 +448,7 @@ func TestQueryWithUniqueCompositeIndex_WithNotEqualFilter_ShouldFetch(t *testing }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -525,7 +525,7 @@ func TestQueryWithUniqueCompositeIndex_WithInForFirstAndEqForRest_ShouldFetchEff }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -580,7 +580,7 @@ func TestQueryWithUniqueCompositeIndex_WithInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(5), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(5), }, }, } @@ -620,7 +620,7 @@ func TestQueryWithUniqueCompositeIndex_WithNotInFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -687,7 +687,7 @@ func TestQueryWithUniqueCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req2, @@ -699,7 +699,7 @@ func TestQueryWithUniqueCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req3, @@ -711,7 +711,7 @@ func TestQueryWithUniqueCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req3), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req4, @@ -723,7 +723,7 @@ func TestQueryWithUniqueCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req4), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req5, @@ -735,7 +735,7 @@ func TestQueryWithUniqueCompositeIndex_WithLikeFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req5), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req6, @@ -787,7 +787,7 @@ func TestQueryWithUniqueCompositeIndex_WithNotLikeFilter_ShouldFetch(t *testing. }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -828,7 +828,7 @@ func TestQueryWithUniqueCompositeIndex_WithNotCaseInsensitiveLikeFilter_ShouldFe }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -857,7 +857,7 @@ func TestQueryWithUniqueCompositeIndex_IfFirstFieldIsNotInFilter_ShouldNotUseInd name } }`, - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(20).WithIndexFetches(0), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(0), }, }, } @@ -967,7 +967,7 @@ func TestQueryWithUniqueCompositeIndex_WithMultipleNilOnFirstFieldAndNilFilter_S }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -1099,7 +1099,7 @@ func TestQueryWithUniqueCompositeIndex_WithMultipleNilOnSecondFieldsAndNilFilter }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } diff --git a/tests/integration/index/query_with_unique_index_only_filter_test.go b/tests/integration/index/query_with_unique_index_only_filter_test.go index 4da1bb63ed..1876c3522d 100644 --- a/tests/integration/index/query_with_unique_index_only_filter_test.go +++ b/tests/integration/index/query_with_unique_index_only_filter_test.go @@ -44,7 +44,7 @@ func TestQueryWithUniqueIndex_WithEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } @@ -81,7 +81,7 @@ func TestQueryWithUniqueIndex_WithGreaterThanFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -119,7 +119,7 @@ func TestQueryWithUniqueIndex_WithGreaterOrEqualFilter_ShouldFetch(t *testing.T) }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -156,7 +156,7 @@ func TestQueryWithUniqueIndex_WithLessThanFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -194,7 +194,7 @@ func TestQueryWithUniqueIndex_WithLessOrEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -239,7 +239,7 @@ func TestQueryWithUniqueIndex_WithNotEqualFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -277,7 +277,7 @@ func TestQueryWithUniqueIndex_WithInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(2), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(2), }, }, } @@ -317,7 +317,7 @@ func TestQueryWithUniqueIndex_WithNotInFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(4).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -380,7 +380,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req1), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req2, @@ -393,7 +393,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req2), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req3, @@ -406,7 +406,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req3), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req4, @@ -418,7 +418,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req4), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req5, @@ -431,7 +431,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req5), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(2).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, testUtils.Request{ Request: req6, @@ -441,7 +441,7 @@ func TestQueryWithUniqueIndex_WithLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req6), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -484,7 +484,7 @@ func TestQueryWithUniqueIndex_WithNotLikeFilter_ShouldFetch(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -528,7 +528,7 @@ func TestQueryWithUniqueIndex_WithNotCaseInsensitiveLikeFilter_ShouldFetch(t *te }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(10), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(10), }, }, } @@ -818,7 +818,7 @@ func TestQueryWithUniqueIndex_WithDateTimeField_ShouldIndex(t *testing.T) { }, testUtils.Request{ Request: makeExplainQuery(req), - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithIndexFetches(1), }, }, } diff --git a/tests/integration/query/json/with_ne_test.go b/tests/integration/query/json/with_ne_test.go index 6a4d619552..dd55b60bf1 100644 --- a/tests/integration/query/json/with_ne_test.go +++ b/tests/integration/query/json/with_ne_test.go @@ -163,3 +163,211 @@ func TestQueryJSON_WithNotEqualFilterWithNullValue_ShouldFilter(t *testing.T) { testUtils.ExecuteTestCase(t, test) } + +func TestQueryJSON_WithNeFilterAgainstNumberField_ShouldFilter(t *testing.T) { + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "age": 48, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "age": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "age": 42, + }, + }, + }, + testUtils.Request{ + Request: `query { + User(filter: {custom: {age: {_ne: 48}}}) { + name + } + }`, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "Andy"}, + }, + }, + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryJSON_WithNeFilterAgainstStringField_ShouldFilter(t *testing.T) { + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "city": "Istanbul", + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "city": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "city": "Lucerne", + }, + }, + }, + testUtils.Request{ + Request: `query { + User(filter: {custom: {city: {_ne: "Istanbul"}}}) { + name + } + }`, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "Andy"}, + }, + }, + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryJSON_WithNeFilterAgainstBooleanField_ShouldFilter(t *testing.T) { + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "verified": true, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "verified": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "verified": false, + }, + }, + }, + testUtils.Request{ + Request: `query { + User(filter: {custom: {verified: {_ne: true}}}) { + name + } + }`, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "Andy"}, + }, + }, + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryJSON_WithNeFilterAgainstNullField_ShouldFilter(t *testing.T) { + test := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{ + Schema: ` + type User { + name: String + custom: JSON + }`, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "John", + "custom": map[string]any{ + "age": 48, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Andy", + "custom": map[string]any{ + "age": nil, + }, + }, + }, + testUtils.CreateDoc{ + DocMap: map[string]any{ + "name": "Shahzad", + "custom": map[string]any{ + "age": 42, + }, + }, + }, + testUtils.Request{ + Request: `query { + User(filter: {custom: {age: {_ne: null}}}) { + name + } + }`, + Results: map[string]any{ + "User": []map[string]any{ + {"name": "Shahzad"}, + {"name": "John"}, + }, + }, + }, + }, + } + testUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/schema/updates/add/field/with_index_test.go b/tests/integration/schema/updates/add/field/with_index_test.go index 3c80f6cede..c0a94ec081 100644 --- a/tests/integration/schema/updates/add/field/with_index_test.go +++ b/tests/integration/schema/updates/add/field/with_index_test.go @@ -95,7 +95,7 @@ func TestSchemaUpdatesAddFieldSimple_WithExistingIndexDocsCreatedAfterPatch(t *t name } }`, - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(1), }, }, } @@ -180,7 +180,7 @@ func TestSchemaUpdatesAddFieldSimple_WithExistingIndexDocsCreatedBeforePatch(t * name } }`, - Asserter: testUtils.NewExplainAsserter().WithFieldFetches(0).WithIndexFetches(1), + Asserter: testUtils.NewExplainAsserter().WithFieldFetches(1).WithIndexFetches(1), }, }, }