diff --git a/client/index.go b/client/index.go index 47b52f00c5..69f0362017 100644 --- a/client/index.go +++ b/client/index.go @@ -37,3 +37,22 @@ type IndexDescription struct { // Fields contains the fields that are being indexed. Fields []IndexedFieldDescription } + +// CollectIndexedFields returns all fields that are indexed by all collection indexes. +func (d CollectionDescription) CollectIndexedFields(schema *SchemaDescription) []FieldDescription { + fieldsMap := make(map[string]bool) + fields := make([]FieldDescription, 0, len(d.Indexes)) + for _, index := range d.Indexes { + for _, field := range index.Fields { + for i := range schema.Fields { + colField := schema.Fields[i] + if field.Name == colField.Name && !fieldsMap[field.Name] { + fieldsMap[field.Name] = true + fields = append(fields, colField) + break + } + } + } + } + return fields +} diff --git a/datastore/errors.go b/datastore/errors.go index b08e6d5e70..b248ce6db8 100644 --- a/datastore/errors.go +++ b/datastore/errors.go @@ -14,6 +14,10 @@ import ( "github.com/sourcenetwork/defradb/errors" ) +const ( + errInvalidStoredValue string = "invalid stored value" +) + // Errors returnable from this package. // // This list is incomplete and undefined errors may also be returned. @@ -26,3 +30,9 @@ var ( // ErrNotFound is an error returned when a block is not found. ErrNotFound = errors.New("blockstore: block not found") ) + +// NewErrInvalidStoredValue returns a new error indicating that the stored +// value in the database is invalid. +func NewErrInvalidStoredValue(inner error) error { + return errors.Wrap(errInvalidStoredValue, inner) +} diff --git a/datastore/prefix_query.go b/datastore/prefix_query.go new file mode 100644 index 0000000000..7150aebe48 --- /dev/null +++ b/datastore/prefix_query.go @@ -0,0 +1,81 @@ +// Copyright 2023 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 datastore + +import ( + "context" + "encoding/json" + + ds "github.com/ipfs/go-datastore" + + "github.com/ipfs/go-datastore/query" +) + +// DeserializePrefix deserializes all elements with the given prefix from the given storage. +// It returns the keys and their corresponding elements. +func DeserializePrefix[T any]( + ctx context.Context, + prefix string, + store ds.Read, +) ([]string, []T, error) { + q, err := store.Query(ctx, query.Query{Prefix: prefix}) + if err != nil { + return nil, nil, err + } + + keys := make([]string, 0) + elements := make([]T, 0) + for res := range q.Next() { + if res.Error != nil { + _ = q.Close() + return nil, nil, res.Error + } + + var element T + err = json.Unmarshal(res.Value, &element) + if err != nil { + _ = q.Close() + return nil, nil, NewErrInvalidStoredValue(err) + } + keys = append(keys, res.Key) + elements = append(elements, element) + } + if err := q.Close(); err != nil { + return nil, nil, err + } + return keys, elements, nil +} + +// FetchKeysForPrefix fetches all keys with the given prefix from the given storage. +func FetchKeysForPrefix( + ctx context.Context, + prefix string, + store ds.Read, +) ([]ds.Key, error) { + q, err := store.Query(ctx, query.Query{Prefix: prefix}) + if err != nil { + return nil, err + } + + keys := make([]ds.Key, 0) + for res := range q.Next() { + if res.Error != nil { + _ = q.Close() + return nil, res.Error + } + keys = append(keys, ds.NewKey(res.Key)) + } + if err = q.Close(); err != nil { + return nil, err + } + + return keys, nil +} diff --git a/db/collection_index.go b/db/collection_index.go index 791817a0a3..f3c1ba2e98 100644 --- a/db/collection_index.go +++ b/db/collection_index.go @@ -17,9 +17,6 @@ import ( "strconv" "strings" - ds "github.com/ipfs/go-datastore" - "github.com/ipfs/go-datastore/query" - "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" @@ -63,7 +60,7 @@ func (db *db) getAllIndexes( ) (map[client.CollectionName][]client.IndexDescription, error) { prefix := core.NewCollectionIndexKey("", "") - deserializedIndexes, err := deserializePrefix[client.IndexDescription](ctx, + keys, indexDescriptions, err := datastore.DeserializePrefix[client.IndexDescription](ctx, prefix.ToString(), txn.Systemstore()) if err != nil { @@ -72,12 +69,15 @@ func (db *db) getAllIndexes( indexes := make(map[client.CollectionName][]client.IndexDescription) - for _, indexRec := range deserializedIndexes { - indexKey, err := core.NewCollectionIndexKeyFromString(indexRec.key) + for i := range keys { + indexKey, err := core.NewCollectionIndexKeyFromString(keys[i]) if err != nil { return nil, NewErrInvalidStoredIndexKey(indexKey.ToString()) } - indexes[indexKey.CollectionName] = append(indexes[indexKey.CollectionName], indexRec.element) + indexes[indexKey.CollectionName] = append( + indexes[indexKey.CollectionName], + indexDescriptions[i], + ) } return indexes, nil @@ -89,16 +89,12 @@ func (db *db) fetchCollectionIndexDescriptions( colName string, ) ([]client.IndexDescription, error) { prefix := core.NewCollectionIndexKey(colName, "") - deserializedIndexes, err := deserializePrefix[client.IndexDescription](ctx, + _, indexDescriptions, err := datastore.DeserializePrefix[client.IndexDescription](ctx, prefix.ToString(), txn.Systemstore()) if err != nil { return nil, err } - indexes := make([]client.IndexDescription, 0, len(deserializedIndexes)) - for _, indexRec := range deserializedIndexes { - indexes = append(indexes, indexRec.element) - } - return indexes, nil + return indexDescriptions, nil } func (c *collection) indexNewDoc(ctx context.Context, txn datastore.Txn, doc *client.Document) error { @@ -115,27 +111,6 @@ func (c *collection) indexNewDoc(ctx context.Context, txn datastore.Txn, doc *cl return nil } -// collectIndexedFields returns all fields that are indexed by all collection indexes. -func (c *collection) collectIndexedFields() []client.FieldDescription { - fieldsMap := make(map[string]client.FieldDescription) - for _, index := range c.indexes { - for _, field := range index.Description().Fields { - for i := range c.desc.Schema.Fields { - colField := c.desc.Schema.Fields[i] - if field.Name == colField.Name { - fieldsMap[field.Name] = colField - break - } - } - } - } - fields := make([]client.FieldDescription, 0, len(fieldsMap)) - for _, field := range fieldsMap { - fields = append(fields, field) - } - return fields -} - func (c *collection) updateIndexedDoc( ctx context.Context, txn datastore.Txn, @@ -145,7 +120,13 @@ func (c *collection) updateIndexedDoc( if err != nil { return err } - oldDoc, err := c.get(ctx, txn, c.getPrimaryKeyFromDocKey(doc.Key()), c.collectIndexedFields(), false) + desc := c.Description() + oldDoc, err := c.get( + ctx, + txn, + c.getPrimaryKeyFromDocKey(doc.Key()), desc.CollectIndexedFields(&desc.Schema), + false, + ) if err != nil { return err } @@ -370,7 +351,7 @@ func (c *collection) dropIndex(ctx context.Context, txn datastore.Txn, indexName func (c *collection) dropAllIndexes(ctx context.Context, txn datastore.Txn) error { prefix := core.NewCollectionIndexKey(c.Name(), "") - keys, err := fetchKeysForPrefix(ctx, prefix.ToString(), txn.Systemstore()) + keys, err := datastore.FetchKeysForPrefix(ctx, prefix.ToString(), txn.Systemstore()) if err != nil { return err } @@ -510,39 +491,3 @@ func generateIndexName(col client.Collection, fields []client.IndexedFieldDescri } return sb.String() } - -type deserializedElement[T any] struct { - key string - element T -} - -func deserializePrefix[T any]( - ctx context.Context, - prefix string, - storage ds.Read, -) ([]deserializedElement[T], error) { - q, err := storage.Query(ctx, query.Query{Prefix: prefix}) - if err != nil { - return nil, NewErrFailedToCreateCollectionQuery(err) - } - - elements := make([]deserializedElement[T], 0) - for res := range q.Next() { - if res.Error != nil { - _ = q.Close() - return nil, res.Error - } - - var element T - err = json.Unmarshal(res.Value, &element) - if err != nil { - _ = q.Close() - return nil, NewErrInvalidStoredIndex(err) - } - elements = append(elements, deserializedElement[T]{key: res.Key, element: element}) - } - if err := q.Close(); err != nil { - return nil, err - } - return elements, nil -} diff --git a/db/fetcher/encoded_doc.go b/db/fetcher/encoded_doc.go index 3e19eb2218..bc22471465 100644 --- a/db/fetcher/encoded_doc.go +++ b/db/fetcher/encoded_doc.go @@ -131,6 +131,24 @@ func Decode(encdoc EncodedDocument) (*client.Document, error) { return doc, nil } +// MergeProperties merges the properties of the given document into this document. +// Existing fields of the current document are overwritten. +func (encdoc *encodedDocument) MergeProperties(other EncodedDocument) { + otherEncDoc, ok := other.(*encodedDocument) + if !ok { + return + } + for field, prop := range otherEncDoc.properties { + encdoc.properties[field] = prop + } + if other.Key() != nil { + encdoc.key = other.Key() + } + if other.SchemaVersionID() != "" { + encdoc.schemaVersionID = other.SchemaVersionID() + } +} + // DecodeToDoc returns a decoded document as a // map of field/value pairs func DecodeToDoc(encdoc EncodedDocument, mapping *core.DocumentMapping, filter bool) (core.Doc, error) { diff --git a/db/fetcher/fetcher.go b/db/fetcher/fetcher.go index 34f05d4f1d..8935e617cc 100644 --- a/db/fetcher/fetcher.go +++ b/db/fetcher/fetcher.go @@ -33,18 +33,22 @@ type ExecInfo struct { DocsFetched uint64 // Number of fields fetched. FieldsFetched uint64 + // Number of indexes fetched. + IndexesFetched uint64 } // Add adds the other ExecInfo to the current ExecInfo. func (s *ExecInfo) Add(other ExecInfo) { s.DocsFetched += other.DocsFetched s.FieldsFetched += other.FieldsFetched + s.IndexesFetched += other.IndexesFetched } // Reset resets the ExecInfo. func (s *ExecInfo) Reset() { s.DocsFetched = 0 s.FieldsFetched = 0 + s.IndexesFetched = 0 } // Fetcher is the interface for collecting documents from the underlying data store. @@ -576,6 +580,8 @@ func (df *DocumentFetcher) fetchNext(ctx context.Context) (EncodedDocument, Exec // keyparts := df.kv.Key.List() // key := keyparts[len(keyparts)-2] + prevExecInfo := df.execInfo + defer func() { df.execInfo.Add(prevExecInfo) }() df.execInfo.Reset() // iterate until we have collected all the necessary kv pairs for the doc // we'll know when were done when either diff --git a/db/fetcher/indexer.go b/db/fetcher/indexer.go new file mode 100644 index 0000000000..da4dc6a580 --- /dev/null +++ b/db/fetcher/indexer.go @@ -0,0 +1,165 @@ +// Copyright 2023 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 fetcher + +import ( + "context" + + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/core" + "github.com/sourcenetwork/defradb/datastore" + "github.com/sourcenetwork/defradb/db/base" + "github.com/sourcenetwork/defradb/planner/mapper" +) + +// 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.CollectionDescription + txn datastore.Txn + indexFilter *mapper.Filter + docFilter *mapper.Filter + doc *encodedDocument + mapping *core.DocumentMapping + indexedField client.FieldDescription + docFields []client.FieldDescription + indexIter indexIterator + indexDataStoreKey core.IndexDataStoreKey + execInfo ExecInfo +} + +var _ Fetcher = (*IndexFetcher)(nil) + +// NewIndexFetcher creates a new IndexFetcher. +func NewIndexFetcher( + docFetcher Fetcher, + indexedFieldDesc client.FieldDescription, + indexFilter *mapper.Filter, +) *IndexFetcher { + return &IndexFetcher{ + docFetcher: docFetcher, + indexedField: indexedFieldDesc, + indexFilter: indexFilter, + } +} + +func (f *IndexFetcher) Init( + ctx context.Context, + txn datastore.Txn, + col *client.CollectionDescription, + fields []client.FieldDescription, + filter *mapper.Filter, + docMapper *core.DocumentMapping, + reverse bool, + showDeleted bool, +) error { + f.col = col + f.docFilter = filter + f.doc = &encodedDocument{} + f.mapping = docMapper + f.txn = txn + + for _, index := range col.Indexes { + if index.Fields[0].Name == f.indexedField.Name { + f.indexDataStoreKey.IndexID = index.ID + break + } + } + + f.indexDataStoreKey.CollectionID = f.col.ID + + for i := range fields { + if fields[i].Name == f.indexedField.Name { + f.docFields = append(fields[:i], fields[i+1:]...) + break + } + } + + iter, err := createIndexIterator(f.indexDataStoreKey, f.indexFilter, &f.execInfo) + if err != nil { + return err + } + f.indexIter = iter + + if f.docFetcher != nil && len(f.docFields) > 0 { + err = f.docFetcher.Init(ctx, f.txn, f.col, f.docFields, f.docFilter, f.mapping, false, false) + } + + return err +} + +func (f *IndexFetcher) Start(ctx context.Context, spans core.Spans) error { + err := f.indexIter.Init(ctx, f.txn.Datastore()) + if err != nil { + return err + } + return nil +} + +func (f *IndexFetcher) FetchNext(ctx context.Context) (EncodedDocument, ExecInfo, error) { + totalExecInfo := f.execInfo + defer func() { f.execInfo.Add(totalExecInfo) }() + f.execInfo.Reset() + for { + f.doc.Reset() + + indexKey, hasValue, err := f.indexIter.Next() + if err != nil { + return nil, ExecInfo{}, err + } + + if !hasValue { + return nil, f.execInfo, nil + } + + property := &encProperty{ + Desc: f.indexedField, + Raw: indexKey.FieldValues[0], + } + + f.doc.key = indexKey.FieldValues[1] + f.doc.properties[f.indexedField] = property + f.execInfo.FieldsFetched++ + + if f.docFetcher != nil && len(f.docFields) > 0 { + targetKey := base.MakeDocKey(*f.col, string(f.doc.key)) + spans := core.NewSpans(core.NewSpan(targetKey, targetKey.PrefixEnd())) + err = f.docFetcher.Start(ctx, spans) + if err != nil { + return nil, ExecInfo{}, err + } + encDoc, execInfo, err := f.docFetcher.FetchNext(ctx) + if err != nil { + return nil, ExecInfo{}, err + } + err = f.docFetcher.Close() + if err != nil { + return nil, ExecInfo{}, err + } + f.execInfo.Add(execInfo) + if encDoc == nil { + continue + } + f.doc.MergeProperties(encDoc) + } else { + f.execInfo.DocsFetched++ + } + return f.doc, f.execInfo, nil + } +} + +func (f *IndexFetcher) Close() error { + if f.indexIter != nil { + return f.indexIter.Close() + } + return nil +} diff --git a/db/fetcher/indexer_iterators.go b/db/fetcher/indexer_iterators.go new file mode 100644 index 0000000000..b563c9b3a3 --- /dev/null +++ b/db/fetcher/indexer_iterators.go @@ -0,0 +1,464 @@ +// Copyright 2023 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 fetcher + +import ( + "bytes" + "context" + "errors" + "strings" + + "github.com/fxamacker/cbor/v2" + + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/connor" + "github.com/sourcenetwork/defradb/core" + "github.com/sourcenetwork/defradb/datastore" + "github.com/sourcenetwork/defradb/planner/mapper" + + "github.com/ipfs/go-datastore/query" +) + +const ( + opEq = "_eq" + opGt = "_gt" + opGe = "_ge" + opLt = "_lt" + opLe = "_le" + opNe = "_ne" + opIn = "_in" + opNin = "_nin" + opLike = "_like" + opNlike = "_nlike" +) + +// indexIterator is an iterator over index keys. +// It is used to iterate over the index keys that match a specific condition. +// For example, iteration over condition _eq and _gt will have completely different logic. +type indexIterator interface { + Init(context.Context, datastore.DSReaderWriter) error + Next() (core.IndexDataStoreKey, bool, error) + Close() error +} + +type queryResultIterator struct { + resultIter query.Results +} + +func (i queryResultIterator) Next() (core.IndexDataStoreKey, bool, error) { + res, hasVal := i.resultIter.NextSync() + if res.Error != nil { + return core.IndexDataStoreKey{}, false, res.Error + } + if !hasVal { + return core.IndexDataStoreKey{}, false, nil + } + key, err := core.NewIndexDataStoreKey(res.Key) + if err != nil { + return core.IndexDataStoreKey{}, false, err + } + return key, true, nil +} + +func (i queryResultIterator) Close() error { + return i.resultIter.Close() +} + +type eqIndexIterator struct { + queryResultIterator + indexKey core.IndexDataStoreKey + filterVal []byte + execInfo *ExecInfo +} + +func (i *eqIndexIterator) Init(ctx context.Context, store datastore.DSReaderWriter) error { + i.indexKey.FieldValues = [][]byte{i.filterVal} + resultIter, err := store.Query(ctx, query.Query{ + Prefix: i.indexKey.ToString(), + KeysOnly: true, + }) + if err != nil { + return err + } + i.resultIter = resultIter + return nil +} + +func (i *eqIndexIterator) Next() (core.IndexDataStoreKey, bool, error) { + key, hasValue, err := i.queryResultIterator.Next() + if hasValue { + i.execInfo.IndexesFetched++ + } + return key, hasValue, err +} + +type inIndexIterator struct { + eqIndexIterator + filterValues [][]byte + nextValIndex int + ctx context.Context + store datastore.DSReaderWriter + hasIterator bool +} + +func newInIndexIterator( + indexKey core.IndexDataStoreKey, + filterValues [][]byte, + execInfo *ExecInfo, +) *inIndexIterator { + return &inIndexIterator{ + eqIndexIterator: eqIndexIterator{ + indexKey: indexKey, + execInfo: execInfo, + }, + filterValues: filterValues, + } +} + +func (i *inIndexIterator) nextIterator() (bool, error) { + if i.nextValIndex > 0 { + err := i.eqIndexIterator.Close() + if err != nil { + return false, err + } + } + + if i.nextValIndex >= len(i.filterValues) { + return false, nil + } + + i.filterVal = i.filterValues[i.nextValIndex] + err := i.eqIndexIterator.Init(i.ctx, i.store) + if err != nil { + return false, err + } + i.nextValIndex++ + return true, nil +} + +func (i *inIndexIterator) Init(ctx context.Context, store datastore.DSReaderWriter) error { + i.ctx = ctx + i.store = store + var err error + i.hasIterator, err = i.nextIterator() + return err +} + +func (i *inIndexIterator) Next() (core.IndexDataStoreKey, bool, error) { + for i.hasIterator { + key, hasValue, err := i.eqIndexIterator.Next() + if err != nil { + return core.IndexDataStoreKey{}, false, err + } + if !hasValue { + i.hasIterator, err = i.nextIterator() + if err != nil { + return core.IndexDataStoreKey{}, false, err + } + continue + } + return key, true, nil + } + return core.IndexDataStoreKey{}, false, nil +} + +func (i *inIndexIterator) Close() error { + return nil +} + +type errorCheckingFilter struct { + matcher indexMatcher + err error +} + +func (f *errorCheckingFilter) Filter(e query.Entry) bool { + if f.err != nil { + return false + } + indexKey, err := core.NewIndexDataStoreKey(e.Key) + if err != nil { + f.err = err + return false + } + res, err := f.matcher.Match(indexKey) + if err != nil { + f.err = err + return false + } + return res +} + +// execInfoIndexMatcherDecorator is a decorator for indexMatcher that counts the number +// of indexes fetched on every call to Match. +type execInfoIndexMatcherDecorator struct { + matcher indexMatcher + execInfo *ExecInfo +} + +func (d *execInfoIndexMatcherDecorator) Match(key core.IndexDataStoreKey) (bool, error) { + d.execInfo.IndexesFetched++ + return d.matcher.Match(key) +} + +type scanningIndexIterator struct { + queryResultIterator + indexKey core.IndexDataStoreKey + matcher indexMatcher + filter errorCheckingFilter + execInfo *ExecInfo +} + +func (i *scanningIndexIterator) Init(ctx context.Context, store datastore.DSReaderWriter) error { + i.filter.matcher = &execInfoIndexMatcherDecorator{matcher: i.matcher, execInfo: i.execInfo} + + iter, err := store.Query(ctx, query.Query{ + Prefix: i.indexKey.ToString(), + KeysOnly: true, + Filters: []query.Filter{&i.filter}, + }) + if err != nil { + return err + } + i.resultIter = iter + + return nil +} + +func (i *scanningIndexIterator) Next() (core.IndexDataStoreKey, bool, error) { + key, hasValue, err := i.queryResultIterator.Next() + if i.filter.err != nil { + return core.IndexDataStoreKey{}, false, i.filter.err + } + return key, hasValue, err +} + +// checks if the stored index value satisfies the condition +type indexMatcher interface { + Match(core.IndexDataStoreKey) (bool, error) +} + +// indexByteValuesMatcher is a filter that compares the index value with a given value. +// It uses bytes.Compare to compare the values and evaluate the result with evalFunc. +type indexByteValuesMatcher struct { + value []byte + // evalFunc receives a result of bytes.Compare + evalFunc func(int) bool +} + +func (m *indexByteValuesMatcher) Match(key core.IndexDataStoreKey) (bool, error) { + res := bytes.Compare(key.FieldValues[0], m.value) + return m.evalFunc(res), nil +} + +// matcher if _ne condition is met +type neIndexMatcher struct { + value []byte +} + +func (m *neIndexMatcher) Match(key core.IndexDataStoreKey) (bool, error) { + return !bytes.Equal(key.FieldValues[0], m.value), nil +} + +// checks if the index value is or is not in the given array +type indexInArrayMatcher struct { + values map[string]bool + isIn bool +} + +func newNinIndexCmp(values [][]byte, isIn bool) *indexInArrayMatcher { + valuesMap := make(map[string]bool) + for _, v := range values { + valuesMap[string(v)] = true + } + return &indexInArrayMatcher{values: valuesMap, isIn: isIn} +} + +func (m *indexInArrayMatcher) Match(key core.IndexDataStoreKey) (bool, error) { + _, found := m.values[string(key.FieldValues[0])] + return found == m.isIn, nil +} + +// checks if the index value satisfies the LIKE condition +type indexLikeMatcher struct { + hasPrefix bool + hasSuffix bool + startAndEnd []string + isLike bool + value string +} + +func newLikeIndexCmp(filterValue string, isLike bool) *indexLikeMatcher { + matcher := &indexLikeMatcher{ + isLike: isLike, + } + if len(filterValue) >= 2 { + if filterValue[0] == '%' { + matcher.hasPrefix = true + filterValue = strings.TrimPrefix(filterValue, "%") + } + if filterValue[len(filterValue)-1] == '%' { + matcher.hasSuffix = true + filterValue = strings.TrimSuffix(filterValue, "%") + } + if !matcher.hasPrefix && !matcher.hasSuffix { + matcher.startAndEnd = strings.Split(filterValue, "%") + } + } + matcher.value = filterValue + + return matcher +} + +func (m *indexLikeMatcher) Match(key core.IndexDataStoreKey) (bool, error) { + var currentVal string + err := cbor.Unmarshal(key.FieldValues[0], ¤tVal) + if err != nil { + return false, err + } + + return m.doesMatch(currentVal) == m.isLike, nil +} + +func (m *indexLikeMatcher) doesMatch(currentVal string) bool { + switch { + case m.hasPrefix && m.hasSuffix: + return strings.Contains(currentVal, m.value) + case m.hasPrefix: + return strings.HasSuffix(currentVal, m.value) + case m.hasSuffix: + return strings.HasPrefix(currentVal, m.value) + // there might be 2 ends only for LIKE with 1 % in the middle "ab%cd" + case len(m.startAndEnd) == 2: + return strings.HasPrefix(currentVal, m.startAndEnd[0]) && + strings.HasSuffix(currentVal, m.startAndEnd[1]) + default: + return m.value == currentVal + } +} + +func createIndexIterator( + indexDataStoreKey core.IndexDataStoreKey, + indexFilterConditions *mapper.Filter, + execInfo *ExecInfo, +) (indexIterator, error) { + var op string + var filterVal any + for _, indexFilterCond := range indexFilterConditions.Conditions { + condMap := indexFilterCond.(map[connor.FilterKey]any) + var key connor.FilterKey + for key, filterVal = range condMap { + break + } + opKey := key.(*mapper.Operator) + op = opKey.Operation + break + } + + switch op { + case opEq, opGt, opGe, opLt, opLe, opNe: + writableValue := client.NewCBORValue(client.LWW_REGISTER, filterVal) + + valueBytes, err := writableValue.Bytes() + if err != nil { + return nil, err + } + + switch op { + case opEq: + return &eqIndexIterator{ + indexKey: indexDataStoreKey, + filterVal: valueBytes, + execInfo: execInfo, + }, nil + case opGt: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: &indexByteValuesMatcher{ + value: valueBytes, + evalFunc: func(res int) bool { return res > 0 }, + }, + execInfo: execInfo, + }, nil + case opGe: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: &indexByteValuesMatcher{ + value: valueBytes, + evalFunc: func(res int) bool { return res > 0 || res == 0 }, + }, + execInfo: execInfo, + }, nil + case opLt: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: &indexByteValuesMatcher{ + value: valueBytes, + evalFunc: func(res int) bool { return res < 0 }, + }, + execInfo: execInfo, + }, nil + case opLe: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: &indexByteValuesMatcher{ + value: valueBytes, + evalFunc: func(res int) bool { return res < 0 || res == 0 }, + }, + execInfo: execInfo, + }, nil + case opNe: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: &neIndexMatcher{ + value: valueBytes, + }, + execInfo: execInfo, + }, nil + } + case opIn, opNin: + inArr, ok := filterVal.([]any) + if !ok { + return nil, errors.New("invalid _in/_nin value") + } + valArr := make([][]byte, 0, len(inArr)) + for _, v := range inArr { + writableValue := client.NewCBORValue(client.LWW_REGISTER, v) + valueBytes, err := writableValue.Bytes() + if err != nil { + return nil, err + } + valArr = append(valArr, valueBytes) + } + if op == opIn { + return newInIndexIterator(indexDataStoreKey, valArr, execInfo), nil + } else { + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: newNinIndexCmp(valArr, false), + execInfo: execInfo, + }, nil + } + case opLike: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: newLikeIndexCmp(filterVal.(string), true), + execInfo: execInfo, + }, nil + case opNlike: + return &scanningIndexIterator{ + indexKey: indexDataStoreKey, + matcher: newLikeIndexCmp(filterVal.(string), false), + execInfo: execInfo, + }, nil + } + + return nil, errors.New("invalid index filter condition") +} diff --git a/db/index.go b/db/index.go index 2c5ea2d6b2..7314bc2a08 100644 --- a/db/index.go +++ b/db/index.go @@ -14,10 +14,6 @@ import ( "context" "time" - ds "github.com/ipfs/go-datastore" - - "github.com/ipfs/go-datastore/query" - "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" @@ -47,7 +43,7 @@ func canConvertIndexFieldValue[T any](val any) bool { func getValidateIndexFieldFunc(kind client.FieldKind) func(any) bool { switch kind { - case client.FieldKind_STRING: + case client.FieldKind_STRING, client.FieldKind_FOREIGN_OBJECT: return canConvertIndexFieldValue[string] case client.FieldKind_INT: return canConvertIndexFieldValue[int64] @@ -179,31 +175,6 @@ func (i *collectionSimpleIndex) Update( return i.Save(ctx, txn, newDoc) } -func fetchKeysForPrefix( - ctx context.Context, - prefix string, - storage ds.Read, -) ([]ds.Key, error) { - q, err := storage.Query(ctx, query.Query{Prefix: prefix}) - if err != nil { - return nil, err - } - - keys := make([]ds.Key, 0) - for res := range q.Next() { - if res.Error != nil { - _ = q.Close() - return nil, res.Error - } - keys = append(keys, ds.NewKey(res.Key)) - } - if err = q.Close(); err != nil { - return nil, err - } - - return keys, nil -} - // RemoveAll remove all artifacts of the index from the storage, i.e. all index // field values for all documents. func (i *collectionSimpleIndex) RemoveAll(ctx context.Context, txn datastore.Txn) error { @@ -211,7 +182,7 @@ func (i *collectionSimpleIndex) RemoveAll(ctx context.Context, txn datastore.Txn prefixKey.CollectionID = i.collection.ID() prefixKey.IndexID = i.desc.ID - keys, err := fetchKeysForPrefix(ctx, prefixKey.ToString(), txn.Datastore()) + keys, err := datastore.FetchKeysForPrefix(ctx, prefixKey.ToString(), txn.Datastore()) if err != nil { return err } diff --git a/db/index_test.go b/db/index_test.go index 62477897f4..5082854455 100644 --- a/db/index_test.go +++ b/db/index_test.go @@ -703,7 +703,7 @@ func TestGetIndexes_IfInvalidIndexIsStored_ReturnError(t *testing.T) { assert.NoError(t, err) _, err = f.getAllIndexes() - assert.ErrorIs(t, err, NewErrInvalidStoredIndex(nil)) + assert.ErrorIs(t, err, datastore.NewErrInvalidStoredValue(nil)) } func TestGetIndexes_IfInvalidIndexKeyIsStored_ReturnError(t *testing.T) { @@ -728,14 +728,15 @@ func TestGetIndexes_IfInvalidIndexKeyIsStored_ReturnError(t *testing.T) { func TestGetIndexes_IfSystemStoreFails_ReturnError(t *testing.T) { f := newIndexTestFixture(t) + testErr := errors.New("test error") + mockedTxn := f.mockTxn() mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything).Unset() - mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything). - Return(nil, errors.New("test error")) + mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything).Return(nil, testErr) _, err := f.getAllIndexes() - assert.ErrorIs(t, err, NewErrFailedToCreateCollectionQuery(nil)) + assert.ErrorIs(t, err, testErr) } func TestGetIndexes_IfSystemStoreFails_ShouldCloseIterator(t *testing.T) { @@ -779,7 +780,7 @@ func TestGetIndexes_IfSystemStoreHasInvalidData_ReturnError(t *testing.T) { mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything).Return(q, nil) _, err := f.getAllIndexes() - assert.ErrorIs(t, err, NewErrInvalidStoredIndex(nil)) + assert.ErrorIs(t, err, datastore.NewErrInvalidStoredValue(nil)) } func TestGetIndexes_IfFailsToReadSeqNumber_ReturnError(t *testing.T) { @@ -853,15 +854,16 @@ func TestGetCollectionIndexes_ShouldReturnListOfCollectionIndexes(t *testing.T) func TestGetCollectionIndexes_IfSystemStoreFails_ReturnError(t *testing.T) { f := newIndexTestFixture(t) + testErr := errors.New("test error") + mockedTxn := f.mockTxn() mockedTxn.MockSystemstore = mocks.NewDSReaderWriter(t) - mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything). - Return(nil, errors.New("test error")) + mockedTxn.MockSystemstore.EXPECT().Query(mock.Anything, mock.Anything).Return(nil, testErr) mockedTxn.EXPECT().Systemstore().Unset() mockedTxn.EXPECT().Systemstore().Return(mockedTxn.MockSystemstore) _, err := f.getCollectionIndexes(usersColName) - assert.ErrorIs(t, err, NewErrFailedToCreateCollectionQuery(nil)) + assert.ErrorIs(t, err, testErr) } func TestGetCollectionIndexes_IfSystemStoreFails_ShouldCloseIterator(t *testing.T) { @@ -902,7 +904,7 @@ func TestGetCollectionIndexes_IfInvalidIndexIsStored_ReturnError(t *testing.T) { assert.NoError(t, err) _, err = f.getCollectionIndexes(usersColName) - assert.ErrorIs(t, err, NewErrInvalidStoredIndex(nil)) + assert.ErrorIs(t, err, datastore.NewErrInvalidStoredValue(nil)) } func TestCollectionGetIndexes_ShouldReturnIndexes(t *testing.T) { @@ -967,7 +969,7 @@ func TestCollectionGetIndexes_IfSystemStoreFails_ReturnError(t *testing.T) { }, { Name: "Query iterator returns invalid value", - ExpectedError: NewErrInvalidStoredIndex(nil), + ExpectedError: datastore.NewErrInvalidStoredValue(nil), GetMockSystemstore: func(t *testing.T) *mocks.DSReaderWriter { store := mocks.NewDSReaderWriter(t) store.EXPECT().Query(mock.Anything, mock.Anything). diff --git a/db/indexed_docs_test.go b/db/indexed_docs_test.go index b62cb992d6..5634686778 100644 --- a/db/indexed_docs_test.go +++ b/db/indexed_docs_test.go @@ -325,7 +325,7 @@ func TestNonUnique_IfSystemStorageHasInvalidIndexDescription_Error(t *testing.T) Return(mocks.NewQueryResultsWithValues(t, []byte("invalid")), nil) err := f.users.WithTxn(mockTxn).Create(f.ctx, doc) - require.ErrorIs(t, err, NewErrInvalidStoredIndex(nil)) + assert.ErrorIs(t, err, datastore.NewErrInvalidStoredValue(nil)) } func TestNonUnique_IfSystemStorageFailsToReadIndexDesc_Error(t *testing.T) { diff --git a/errors/defraError.go b/errors/defraError.go index 2281add30e..2f05f1131d 100644 --- a/errors/defraError.go +++ b/errors/defraError.go @@ -58,13 +58,12 @@ func (e *defraError) Error() string { } func (e *defraError) Is(other error) bool { - switch otherTyped := other.(type) { - case *defraError: - return e.message == otherTyped.message - default: - otherString := other.Error() - return e.message == otherString || e.Error() == otherString || errors.Is(e.inner, other) + var otherDefraError *defraError + if errors.As(other, &otherDefraError) { + return e.message == otherDefraError.message } + otherString := other.Error() + return e.message == otherString || e.Error() == otherString || errors.Is(e.inner, other) } func (e *defraError) Unwrap() error { diff --git a/planner/datasource.go b/planner/datasource.go index afcfbab3ce..862f43bd33 100644 --- a/planner/datasource.go +++ b/planner/datasource.go @@ -11,11 +11,7 @@ package planner import ( - "encoding/json" - "github.com/sourcenetwork/defradb/client" - "github.com/sourcenetwork/defradb/core" - "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/planner/mapper" ) @@ -35,13 +31,13 @@ func (p *Planner) getSource(parsed *mapper.Select) (planSource, error) { return p.getCollectionScanPlan(parsed) } -func (p *Planner) getCollectionScanPlan(parsed *mapper.Select) (planSource, error) { - colDesc, err := p.getCollectionDesc(parsed.CollectionName) +func (p *Planner) getCollectionScanPlan(mapperSelect *mapper.Select) (planSource, error) { + col, err := p.db.GetCollectionByName(p.ctx, mapperSelect.CollectionName) if err != nil { return planSource{}, err } - scan, err := p.Scan(parsed) + scan, err := p.Scan(mapperSelect, col.Description()) if err != nil { return planSource{}, err } @@ -49,30 +45,7 @@ func (p *Planner) getCollectionScanPlan(parsed *mapper.Select) (planSource, erro return planSource{ plan: scan, info: sourceInfo{ - collectionDescription: colDesc, + collectionDescription: col.Description(), }, }, nil } - -func (p *Planner) getCollectionDesc(name string) (client.CollectionDescription, error) { - collectionKey := core.NewCollectionKey(name) - var desc client.CollectionDescription - schemaVersionIdBytes, err := p.txn.Systemstore().Get(p.ctx, collectionKey.ToDS()) - if err != nil { - return desc, errors.Wrap("failed to get collection description", err) - } - - schemaVersionId := string(schemaVersionIdBytes) - schemaVersionKey := core.NewCollectionSchemaVersionKey(schemaVersionId) - buf, err := p.txn.Systemstore().Get(p.ctx, schemaVersionKey.ToDS()) - if err != nil { - return desc, err - } - - err = json.Unmarshal(buf, &desc) - if err != nil { - return desc, err - } - - return desc, nil -} diff --git a/planner/explain.go b/planner/explain.go index 560063b4ba..07f96f9b0a 100644 --- a/planner/explain.go +++ b/planner/explain.go @@ -286,10 +286,10 @@ func buildSimpleExplainGraph(source planNode) (map[string]any, error) { // // Note: Can only be called once the entire plan has been executed. func collectExecuteExplainInfo(executedPlan planNode) (map[string]any, error) { - excuteExplainInfo := map[string]any{} + executeExplainInfo := map[string]any{} if executedPlan == nil { - return excuteExplainInfo, nil + return executeExplainInfo, nil } switch executedNode := executedPlan.(type) { @@ -303,16 +303,16 @@ func collectExecuteExplainInfo(executedPlan planNode) (map[string]any, error) { multiChildExplainGraph = append(multiChildExplainGraph, childExplainGraph) } explainNodeLabelTitle := strcase.ToLowerCamel(executedNode.Kind()) - excuteExplainInfo[explainNodeLabelTitle] = multiChildExplainGraph + executeExplainInfo[explainNodeLabelTitle] = multiChildExplainGraph case explainablePlanNode: - excuteExplainBuilder, err := executedNode.Explain(request.ExecuteExplain) + executeExplainBuilder, err := executedNode.Explain(request.ExecuteExplain) if err != nil { return nil, err } - if excuteExplainBuilder == nil { - excuteExplainBuilder = map[string]any{} + if executeExplainBuilder == nil { + executeExplainBuilder = map[string]any{} } if next := executedNode.Source(); next != nil && next.Kind() != topLevelNodeKind { @@ -321,21 +321,21 @@ func collectExecuteExplainInfo(executedPlan planNode) (map[string]any, error) { return nil, err } for key, value := range nextExplainGraph { - excuteExplainBuilder[key] = value + executeExplainBuilder[key] = value } } explainNodeLabelTitle := strcase.ToLowerCamel(executedNode.Kind()) - excuteExplainInfo[explainNodeLabelTitle] = excuteExplainBuilder + executeExplainInfo[explainNodeLabelTitle] = executeExplainBuilder default: var err error - excuteExplainInfo, err = collectExecuteExplainInfo(executedPlan.Source()) + executeExplainInfo, err = collectExecuteExplainInfo(executedPlan.Source()) if err != nil { return nil, err } } - return excuteExplainInfo, nil + return executeExplainInfo, nil } // executeAndExplainRequest executes the plan graph gathering the information/datapoints diff --git a/planner/filter/complex.go b/planner/filter/complex.go index 098caefc9c..acc2de4883 100644 --- a/planner/filter/complex.go +++ b/planner/filter/complex.go @@ -17,7 +17,7 @@ import ( // IsComplex returns true if the provided filter is complex. // A filter is considered complex if it contains a relation -// object withing an _or operator not necessarily being +// object withing an _or or _not operator not necessarily being // its direct child. func IsComplex(filter *mapper.Filter) bool { if filter == nil { diff --git a/planner/filter/copy_field.go b/planner/filter/copy_field.go index 59f7db3471..70b5dc2956 100644 --- a/planner/filter/copy_field.go +++ b/planner/filter/copy_field.go @@ -14,18 +14,22 @@ import ( "github.com/sourcenetwork/defradb/planner/mapper" ) -// copyField copies the given field from the provided filter. +// CopyField copies the given field from the provided filter. +// Multiple fields can be passed to copy related objects with a certain field. // The result filter preserves the structure of the original filter. -func copyField(filter *mapper.Filter, field mapper.Field) *mapper.Filter { - if filter == nil { +func CopyField(filter *mapper.Filter, fields ...mapper.Field) *mapper.Filter { + if filter == nil || len(fields) == 0 { return nil } - conditionKey := &mapper.PropertyIndex{ - Index: field.Index, + var conditionKeys []*mapper.PropertyIndex + for _, field := range fields { + conditionKeys = append(conditionKeys, &mapper.PropertyIndex{ + Index: field.Index, + }) } resultFilter := &mapper.Filter{} - conditionMap := traverseFilterByProperty(conditionKey, filter.Conditions, false) + conditionMap := traverseFilterByProperty(conditionKeys, filter.Conditions, false) if len(conditionMap) > 0 { resultFilter.Conditions = conditionMap return resultFilter @@ -34,7 +38,7 @@ func copyField(filter *mapper.Filter, field mapper.Field) *mapper.Filter { } func traverseFilterByProperty( - key *mapper.PropertyIndex, + keys []*mapper.PropertyIndex, conditions map[connor.FilterKey]any, shouldDelete bool, ) map[connor.FilterKey]any { @@ -43,11 +47,20 @@ func traverseFilterByProperty( result = make(map[connor.FilterKey]any) } for targetKey, clause := range conditions { - if targetKey.Equal(key) { - if shouldDelete { - delete(result, targetKey) + if targetKey.Equal(keys[0]) { + if len(keys) > 1 { + related := traverseFilterByProperty(keys[1:], clause.(map[connor.FilterKey]any), shouldDelete) + if shouldDelete && len(related) == 0 { + delete(result, targetKey) + } else if len(related) > 0 && !shouldDelete { + result[keys[0]] = clause + } } else { - result[key] = clause + if shouldDelete { + delete(result, targetKey) + } else { + result[keys[0]] = clause + } } } else if opKey, isOpKey := targetKey.(*mapper.Operator); isOpKey { clauseArr, isArr := clause.([]any) @@ -58,13 +71,15 @@ func traverseFilterByProperty( if !ok { continue } - compoundCond := traverseFilterByProperty(key, elementMap, shouldDelete) + compoundCond := traverseFilterByProperty(keys, elementMap, shouldDelete) if len(compoundCond) > 0 { resultArr = append(resultArr, compoundCond) } } if len(resultArr) > 0 { result[opKey] = resultArr + } else if shouldDelete { + delete(result, opKey) } } } diff --git a/planner/filter/copy_field_test.go b/planner/filter/copy_field_test.go index d3ec10cf62..1714db55b6 100644 --- a/planner/filter/copy_field_test.go +++ b/planner/filter/copy_field_test.go @@ -13,6 +13,7 @@ import ( "testing" "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/connor" "github.com/sourcenetwork/defradb/planner/mapper" "github.com/stretchr/testify/assert" @@ -21,7 +22,7 @@ import ( func TestCopyField(t *testing.T) { tests := []struct { name string - inputField mapper.Field + inputField []mapper.Field inputFilter map[string]any expectedFilter map[string]any }{ @@ -31,7 +32,7 @@ func TestCopyField(t *testing.T) { "name": m("_eq", "John"), "age": m("_gt", 55), }, - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: m("age", m("_gt", 55)), }, { @@ -40,7 +41,7 @@ func TestCopyField(t *testing.T) { m("name", m("_eq", "John")), m("age", m("_gt", 55)), ), - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: r("_and", m("age", m("_gt", 55)), ), @@ -59,7 +60,7 @@ func TestCopyField(t *testing.T) { m("age", m("_lt", 55)), ), ), - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: r("_and", r("_or", r("_and", @@ -71,13 +72,48 @@ func TestCopyField(t *testing.T) { ), ), }, + { + name: "field of related object", + inputFilter: r("_and", + r("_or", + r("_and", + m("published", m("rating", m("_gt", 4.0))), + m("age", m("_gt", 30)), + ), + ), + m("published", m("genre", m("_eq", "Comedy"))), + m("name", m("_eq", "John")), + ), + inputField: []mapper.Field{{Index: authorPublishedInd}, {Index: bookRatingInd}}, + expectedFilter: r("_and", + r("_or", + r("_and", + m("published", m("rating", m("_gt", 4.0))), + ), + ), + ), + }, + { + name: "field of related object (deeper)", + inputFilter: r("_and", + m("published", m("rating", m("_gt", 4.0))), + m("age", m("_gt", 30)), + m("published", m("stores", m("address", m("_eq", "123 Main St")))), + m("published", m("genre", m("_eq", "Comedy"))), + m("name", m("_eq", "John")), + ), + inputField: []mapper.Field{{Index: authorPublishedInd}, {Index: bookStoresInd}, {Index: storeAddressInd}}, + expectedFilter: r("_and", + m("published", m("stores", m("address", m("_eq", "123 Main St")))), + ), + }, } mapping := getDocMapping() for _, test := range tests { t.Run(test.name, func(t *testing.T) { inputFilter := mapper.ToFilter(request.Filter{Conditions: test.inputFilter}, mapping) - actualFilter := copyField(inputFilter, test.inputField) + actualFilter := CopyField(inputFilter, test.inputField...) expectedFilter := mapper.ToFilter(request.Filter{Conditions: test.expectedFilter}, mapping) AssertEqualFilterMap(t, expectedFilter.Conditions, actualFilter.Conditions) }) @@ -85,6 +121,15 @@ func TestCopyField(t *testing.T) { } func TestCopyFieldOfNullFilter(t *testing.T) { - actualFilter := copyField(nil, mapper.Field{Index: 1}) + actualFilter := CopyField(nil, mapper.Field{Index: 1}) + assert.Nil(t, actualFilter) +} + +func TestCopyFieldWithNoFieldGiven(t *testing.T) { + filter := mapper.NewFilter() + filter.Conditions = map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 0}: &mapper.Operator{Operation: "_eq"}, + } + actualFilter := CopyField(filter) assert.Nil(t, actualFilter) } diff --git a/planner/filter/copy_test.go b/planner/filter/copy_test.go index ccb471c2b6..a45d368964 100644 --- a/planner/filter/copy_test.go +++ b/planner/filter/copy_test.go @@ -23,20 +23,20 @@ func TestCopyFilter(t *testing.T) { return map[connor.FilterKey]any{ &mapper.Operator{Operation: "_or"}: []any{ map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 0}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: authorNameInd}: map[connor.FilterKey]any{ &mapper.Operator{Operation: "_eq"}: "Some name", }, }, map[connor.FilterKey]any{ &mapper.Operator{Operation: "_and"}: []any{ map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: authorAgeInd}: map[connor.FilterKey]any{ &mapper.Operator{Operation: "_gt"}: 64, }, }, map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: authorPublishedInd}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: bookRatingInd}: map[connor.FilterKey]any{ &mapper.Operator{Operation: "_gt"}: 4.8, }, }, @@ -46,13 +46,13 @@ func TestCopyFilter(t *testing.T) { map[connor.FilterKey]any{ &mapper.Operator{Operation: "_and"}: []any{ map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: authorAgeInd}: map[connor.FilterKey]any{ &mapper.Operator{Operation: "_lt"}: 64, }, }, map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 2}: map[connor.FilterKey]any{ - &mapper.PropertyIndex{Index: 1}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: authorPublishedInd}: map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: bookRatingInd}: map[connor.FilterKey]any{ &mapper.Operator{Operation: "_lt"}: 4.8, }, }, diff --git a/planner/filter/extract_properties.go b/planner/filter/extract_properties.go new file mode 100644 index 0000000000..4c3e6bb0be --- /dev/null +++ b/planner/filter/extract_properties.go @@ -0,0 +1,78 @@ +// Copyright 2023 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/connor" + "github.com/sourcenetwork/defradb/planner/mapper" +) + +// Property represents a single field and is being filtered on. +// It contains the index of the field in the core.DocumentMapping +// as well as index -> Property map of the fields in case the field is an object. +type Property struct { + Index int + Fields map[int]Property +} + +func (p Property) IsRelation() bool { + return len(p.Fields) > 0 +} + +func mergeProps(p1, p2 Property) Property { + if p1.Index == 0 { + p1.Index = p2.Index + } + if p1.Fields == nil { + p1.Fields = p2.Fields + } else { + for k, v := range p2.Fields { + p1.Fields[k] = mergeProps(p1.Fields[k], v) + } + } + return p1 +} + +// ExtractProperties runs through the filter and returns a index -> Property map of the fields +// being filtered on. +func ExtractProperties(conditions map[connor.FilterKey]any) map[int]Property { + properties := map[int]Property{} + for k, v := range conditions { + switch typedKey := k.(type) { + case *mapper.PropertyIndex: + prop := properties[typedKey.Index] + prop.Index = typedKey.Index + relatedProps := ExtractProperties(v.(map[connor.FilterKey]any)) + properties[typedKey.Index] = mergeProps(prop, Property{Fields: relatedProps}) + case *mapper.Operator: + if typedKey.Operation == request.FilterOpAnd || typedKey.Operation == request.FilterOpOr { + compoundContent := v.([]any) + for _, compoundFilter := range compoundContent { + props := ExtractProperties(compoundFilter.(map[connor.FilterKey]any)) + for _, prop := range props { + existingProp := properties[prop.Index] + properties[prop.Index] = mergeProps(existingProp, prop) + } + } + } else if typedKey.Operation == request.FilterOpNot { + props := ExtractProperties(v.(map[connor.FilterKey]any)) + for _, prop := range props { + existingProp := properties[prop.Index] + properties[prop.Index] = mergeProps(existingProp, prop) + } + } + } + } + if len(properties) == 0 { + return nil + } + return properties +} diff --git a/planner/filter/extract_properties_test.go b/planner/filter/extract_properties_test.go new file mode 100644 index 0000000000..c90dbe85b0 --- /dev/null +++ b/planner/filter/extract_properties_test.go @@ -0,0 +1,115 @@ +// Copyright 2023 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 ( + "reflect" + "testing" + + "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/planner/mapper" + + "github.com/stretchr/testify/assert" +) + +func TestExtractProperties(t *testing.T) { + tests := []struct { + name string + inputFilter map[string]any + expectedFilter map[int]Property + }{ + { + name: "no nesting", + inputFilter: map[string]any{ + "name": m("_eq", "John"), + "age": m("_gt", 55), + }, + expectedFilter: map[int]Property{ + authorNameInd: {Index: authorNameInd}, + authorAgeInd: {Index: authorAgeInd}, + }, + }, + { + name: "within _and, _or and _not", + inputFilter: r("_or", + m("name", m("_eq", "John")), + r("_and", + m("age", m("_gt", 55)), + m("_not", + r("_or", + m("verified", m("_eq", true)), + ), + ), + ), + ), + expectedFilter: map[int]Property{ + authorNameInd: {Index: authorNameInd}, + authorAgeInd: {Index: authorAgeInd}, + authorVerifiedInd: {Index: authorVerifiedInd}, + }, + }, + { + name: "related field", + inputFilter: r("_or", + m("name", m("_eq", "John")), + m("published", m("genre", m("_eq", "Comedy"))), + ), + expectedFilter: map[int]Property{ + authorNameInd: {Index: authorNameInd}, + authorPublishedInd: { + Index: authorPublishedInd, + Fields: map[int]Property{bookGenreInd: {Index: bookGenreInd}}, + }, + }, + }, + { + name: "several related field with deeper nesting", + inputFilter: r("_or", + m("name", m("_eq", "John")), + m("published", m("genre", m("_eq", "Comedy"))), + m("published", m("rating", m("_gt", 55))), + m("published", m("stores", m("name", m("_eq", "Amazon")))), + m("published", m("stores", m("address", m("_gt", "5th Avenue")))), + ), + expectedFilter: map[int]Property{ + authorNameInd: {Index: authorNameInd}, + authorPublishedInd: { + Index: authorPublishedInd, + Fields: map[int]Property{ + bookGenreInd: {Index: bookGenreInd}, + bookRatingInd: {Index: bookRatingInd}, + bookStoresInd: { + Index: bookStoresInd, + Fields: map[int]Property{ + storeNameInd: {Index: storeNameInd}, + storeAddressInd: {Index: storeAddressInd}, + }, + }, + }, + }, + }, + }, + } + + mapping := getDocMapping() + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + inputFilter := mapper.ToFilter(request.Filter{Conditions: test.inputFilter}, mapping) + actualFilter := ExtractProperties(inputFilter.Conditions) + reflect.DeepEqual(test.expectedFilter, actualFilter) + assert.Equal(t, test.expectedFilter, actualFilter) + }) + } +} + +func TestExtractPropertiesOfNullFilter(t *testing.T) { + actualFilter := CopyField(nil, mapper.Field{Index: 1}) + assert.Nil(t, actualFilter) +} diff --git a/planner/filter/remove_field.go b/planner/filter/remove_field.go index 5c80ffc96c..5e9f2f532e 100644 --- a/planner/filter/remove_field.go +++ b/planner/filter/remove_field.go @@ -14,13 +14,17 @@ import ( ) // RemoveField removes the given field from the provided filter. -func RemoveField(filter *mapper.Filter, field mapper.Field) { - if filter == nil { +// Multiple fields can be passed to remove related objects with a certain field. +func RemoveField(filter *mapper.Filter, fields ...mapper.Field) { + if filter == nil || len(fields) == 0 { return } - conditionKey := &mapper.PropertyIndex{ - Index: field.Index, + var conditionKeys []*mapper.PropertyIndex + for _, field := range fields { + conditionKeys = append(conditionKeys, &mapper.PropertyIndex{ + Index: field.Index, + }) } - traverseFilterByProperty(conditionKey, filter.Conditions, true) + traverseFilterByProperty(conditionKeys, filter.Conditions, true) } diff --git a/planner/filter/remove_field_test.go b/planner/filter/remove_field_test.go index 2b6e8cdd3a..8a34999e60 100644 --- a/planner/filter/remove_field_test.go +++ b/planner/filter/remove_field_test.go @@ -13,13 +13,14 @@ import ( "testing" "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/connor" "github.com/sourcenetwork/defradb/planner/mapper" ) func TestRemoveFieldFromFilter(t *testing.T) { tests := []struct { name string - inputField mapper.Field + inputField []mapper.Field inputFilter map[string]any expectedFilter map[string]any }{ @@ -29,7 +30,7 @@ func TestRemoveFieldFromFilter(t *testing.T) { "name": m("_eq", "John"), "age": m("_gt", 55), }, - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: m("name", m("_eq", "John")), }, { @@ -38,7 +39,7 @@ func TestRemoveFieldFromFilter(t *testing.T) { m("name", m("_eq", "John")), m("age", m("_gt", 55)), ), - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: r("_and", m("name", m("_eq", "John")), ), @@ -57,7 +58,7 @@ func TestRemoveFieldFromFilter(t *testing.T) { m("age", m("_lt", 55)), ), ), - inputField: mapper.Field{Index: 1}, // age + inputField: []mapper.Field{{Index: authorAgeInd}}, expectedFilter: r("_and", r("_or", r("_and", @@ -69,13 +70,69 @@ func TestRemoveFieldFromFilter(t *testing.T) { ), ), }, + { + name: "remove _or/_and if only element", + inputFilter: r("_and", + r("_or", + r("_and", + m("age", m("_gt", 30)), + ), + ), + r("_or", + m("age", m("_lt", 55)), + ), + m("name", m("_eq", "Islam")), + ), + inputField: []mapper.Field{{Index: authorAgeInd}}, + expectedFilter: r("_and", + m("name", m("_eq", "Islam")), + ), + }, + { + name: "field of related object", + inputFilter: r("_and", + r("_or", + r("_and", + m("published", m("rating", m("_gt", 4.0))), + m("age", m("_gt", 30)), + ), + ), + m("published", m("genre", m("_eq", "Comedy"))), + m("name", m("_eq", "John")), + ), + inputField: []mapper.Field{{Index: authorPublishedInd}, {Index: bookRatingInd}}, + expectedFilter: r("_and", + r("_or", + r("_and", + m("age", m("_gt", 30)), + ), + ), + m("published", m("genre", m("_eq", "Comedy"))), + m("name", m("_eq", "John")), + ), + }, + { + name: "field of related object (deeper)", + inputFilter: r("_and", + m("age", m("_gt", 30)), + m("published", m("stores", m("address", m("_eq", "123 Main St")))), + m("published", m("stores", m("name", m("_eq", "Barnes & Noble")))), + m("published", m("genre", m("_eq", "Comedy"))), + ), + inputField: []mapper.Field{{Index: authorPublishedInd}, {Index: bookStoresInd}, {Index: storeAddressInd}}, + expectedFilter: r("_and", + m("age", m("_gt", 30)), + m("published", m("stores", m("name", m("_eq", "Barnes & Noble")))), + m("published", m("genre", m("_eq", "Comedy"))), + ), + }, } mapping := getDocMapping() for _, test := range tests { t.Run(test.name, func(t *testing.T) { inputFilter := mapper.ToFilter(request.Filter{Conditions: test.inputFilter}, mapping) - RemoveField(inputFilter, test.inputField) + RemoveField(inputFilter, test.inputField...) expectedFilter := mapper.ToFilter(request.Filter{Conditions: test.expectedFilter}, mapping) AssertEqualFilterMap(t, expectedFilter.Conditions, inputFilter.Conditions) }) @@ -85,3 +142,17 @@ func TestRemoveFieldFromFilter(t *testing.T) { func TestRemoveFieldFromNullFilter(t *testing.T) { RemoveField(nil, mapper.Field{Index: 1}) } + +func TestRemoveFieldWithNoFieldGiven(t *testing.T) { + getFilter := func() *mapper.Filter { + f := mapper.NewFilter() + f.Conditions = map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 0}: &mapper.Operator{Operation: "_eq"}, + } + return f + } + f := getFilter() + RemoveField(f) + + AssertEqualFilter(t, getFilter(), f) +} diff --git a/planner/filter/split.go b/planner/filter/split.go index bba822145a..1ef153746b 100644 --- a/planner/filter/split.go +++ b/planner/filter/split.go @@ -27,8 +27,12 @@ func SplitByField(filter *mapper.Filter, field mapper.Field) (*mapper.Filter, *m return nil, nil } - splitF := copyField(filter, field) + splitF := CopyField(filter, field) RemoveField(filter, field) + if len(filter.Conditions) == 0 { + filter = nil + } + return filter, splitF } diff --git a/planner/filter/split_test.go b/planner/filter/split_test.go index 1bcbecffb7..86fbb0b44a 100644 --- a/planner/filter/split_test.go +++ b/planner/filter/split_test.go @@ -32,10 +32,28 @@ func TestSplitFilter(t *testing.T) { "name": m("_eq", "John"), "age": m("_gt", 55), }, - inputField: mapper.Field{Index: 1}, // age + inputField: mapper.Field{Index: authorAgeInd}, expectedFilter1: m("name", m("_eq", "John")), expectedFilter2: m("age", m("_gt", 55)), }, + { + name: "the only field", + inputFilter: map[string]any{ + "age": m("_gt", 55), + }, + inputField: mapper.Field{Index: authorAgeInd}, + expectedFilter1: nil, + expectedFilter2: m("age", m("_gt", 55)), + }, + { + name: "no field to delete", + inputFilter: map[string]any{ + "name": m("_eq", "John"), + }, + inputField: mapper.Field{Index: authorAgeInd}, + expectedFilter1: m("name", m("_eq", "John")), + expectedFilter2: nil, + }, } mapping := getDocMapping() @@ -45,14 +63,18 @@ func TestSplitFilter(t *testing.T) { actualFilter1, actualFilter2 := SplitByField(inputFilter, test.inputField) expectedFilter1 := mapper.ToFilter(request.Filter{Conditions: test.expectedFilter1}, mapping) expectedFilter2 := mapper.ToFilter(request.Filter{Conditions: test.expectedFilter2}, mapping) - AssertEqualFilterMap(t, expectedFilter1.Conditions, actualFilter1.Conditions) - AssertEqualFilterMap(t, expectedFilter2.Conditions, actualFilter2.Conditions) + if expectedFilter1 != nil || actualFilter1 != nil { + AssertEqualFilterMap(t, expectedFilter1.Conditions, actualFilter1.Conditions) + } + if expectedFilter2 != nil || actualFilter2 != nil { + AssertEqualFilterMap(t, expectedFilter2.Conditions, actualFilter2.Conditions) + } }) } } func TestSplitNullFilter(t *testing.T) { - actualFilter1, actualFilter2 := SplitByField(nil, mapper.Field{Index: 1}) + actualFilter1, actualFilter2 := SplitByField(nil, mapper.Field{Index: authorAgeInd}) assert.Nil(t, actualFilter1) assert.Nil(t, actualFilter2) } diff --git a/planner/filter/unwrap_relation.go b/planner/filter/unwrap_relation.go new file mode 100644 index 0000000000..aa1be2e25d --- /dev/null +++ b/planner/filter/unwrap_relation.go @@ -0,0 +1,86 @@ +// Copyright 2023 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/connor" + "github.com/sourcenetwork/defradb/planner/mapper" +) + +// UnwrapRelation runs through the filter and returns a new filter with only the +// fields of a given relation object +// Example: +// +// { +// "published": { +// "rating": { +// "_gt": 4.0 +// } +// } +// } +// +// with given "published" field will return +// +// { +// "rating": { +// "_gt": 4.0 +// } +// } +func UnwrapRelation(filter *mapper.Filter, field mapper.Field) *mapper.Filter { + if filter == nil { + return nil + } + conditionKey := &mapper.PropertyIndex{ + Index: field.Index, + } + + resultFilter := &mapper.Filter{} + conditionMap := traverseFilterAndExtract(conditionKey, filter.Conditions, false) + if len(conditionMap) > 0 { + resultFilter.Conditions = conditionMap + return resultFilter + } + return nil +} + +func traverseFilterAndExtract( + key *mapper.PropertyIndex, + conditions map[connor.FilterKey]any, + shouldDelete bool, +) map[connor.FilterKey]any { + result := make(map[connor.FilterKey]any) + for targetKey, clause := range conditions { + if targetKey.Equal(key) { + clauseMap := clause.(map[connor.FilterKey]any) + for k, v := range clauseMap { + result[k] = v + } + } else if opKey, isOpKey := targetKey.(*mapper.Operator); isOpKey { + clauseArr, isArr := clause.([]any) + if isArr { + resultArr := make([]any, 0) + for _, elementClause := range clauseArr { + elementMap, ok := elementClause.(map[connor.FilterKey]any) + if !ok { + continue + } + compoundCond := traverseFilterAndExtract(key, elementMap, shouldDelete) + if len(compoundCond) > 0 { + resultArr = append(resultArr, compoundCond) + } + } + if len(resultArr) > 0 { + result[opKey] = resultArr + } + } + } + } + return result +} diff --git a/planner/filter/unwrap_relation_test.go b/planner/filter/unwrap_relation_test.go new file mode 100644 index 0000000000..a7446f9d30 --- /dev/null +++ b/planner/filter/unwrap_relation_test.go @@ -0,0 +1,99 @@ +// Copyright 2023 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/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/connor" + "github.com/sourcenetwork/defradb/planner/mapper" + + "github.com/stretchr/testify/assert" +) + +func TestUnwrapRelation(t *testing.T) { + tests := []struct { + name string + inputFilter map[string]any + expectedFilter map[string]any + }{ + { + name: "simple", + inputFilter: m("published", m("rating", m("_gt", 4.0))), + expectedFilter: m("rating", m("_gt", 4.0)), + }, + { + name: "no relation object", + inputFilter: map[string]any{ + "name": m("_eq", "John"), + "age": m("_gt", 55), + }, + expectedFilter: nil, + }, + { + name: "within _or and _and", + inputFilter: r("_and", + r("_or", + r("_and", + m("name", m("_eq", "John")), + m("age", m("_gt", 30)), + m("published", m("stores", m("address", m("_eq", "123 Main St")))), + m("published", m("rating", m("_gt", 4.0))), + ), + ), + r("_or", + m("published", m("stores", m("address", m("_eq", "2 Ave")))), + ), + m("published", m("genre", m("_eq", "Comedy"))), + ), + expectedFilter: r("_and", + r("_or", + r("_and", + m("stores", m("address", m("_eq", "123 Main St"))), + m("rating", m("_gt", 4.0)), + ), + ), + r("_or", + m("stores", m("address", m("_eq", "2 Ave"))), + ), + m("genre", m("_eq", "Comedy")), + ), + }, + } + + mapping := getDocMapping() + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + inputFilter := mapper.ToFilter(request.Filter{Conditions: test.inputFilter}, mapping) + actualFilter := UnwrapRelation(inputFilter, mapper.Field{Index: authorPublishedInd}) + childMapping := mapping.ChildMappings[authorPublishedInd] + expectedFilter := mapper.ToFilter(request.Filter{Conditions: test.expectedFilter}, childMapping) + if expectedFilter == nil && actualFilter == nil { + return + } + AssertEqualFilterMap(t, expectedFilter.Conditions, actualFilter.Conditions) + }) + } +} + +func TestUnwrapRelationOfNullFilter(t *testing.T) { + actualFilter := CopyField(nil, mapper.Field{Index: 1}) + assert.Nil(t, actualFilter) +} + +func TestUnwrapRelationWithNoFieldGiven(t *testing.T) { + filter := mapper.NewFilter() + filter.Conditions = map[connor.FilterKey]any{ + &mapper.PropertyIndex{Index: 0}: &mapper.Operator{Operation: "_eq"}, + } + actualFilter := CopyField(filter) + assert.Nil(t, actualFilter) +} diff --git a/planner/filter/util_test.go b/planner/filter/util_test.go index e8860081c8..19b367172c 100644 --- a/planner/filter/util_test.go +++ b/planner/filter/util_test.go @@ -130,11 +130,55 @@ func r(op string, vals ...any) map[string]any { return m(op, vals) } +const ( + authorNameInd = iota + authorAgeInd + authorPublishedInd + authorVerifiedInd + authorNumFields +) + +const ( + bookRatingInd = iota + bookGenreInd + bookNameInd + bookStoresInd + bookNumFields +) + +const ( + storeAddressInd = iota + storeNameInd + storeNumFields +) + func getDocMapping() *core.DocumentMapping { + bookChildMappings := make([]*core.DocumentMapping, bookNumFields) + bookChildMappings[bookStoresInd] = &core.DocumentMapping{ + IndexesByName: map[string][]int{ + "address": {storeAddressInd}, + "name": {storeNameInd}, + }, + } + + authorChildMappings := make([]*core.DocumentMapping, authorNumFields) + authorChildMappings[authorPublishedInd] = &core.DocumentMapping{ + IndexesByName: map[string][]int{ + "rating": {bookRatingInd}, + "genre": {bookGenreInd}, + "name": {bookNameInd}, + "stores": {bookStoresInd}, + }, + ChildMappings: bookChildMappings, + } + return &core.DocumentMapping{ - IndexesByName: map[string][]int{"name": {0}, "age": {1}, "published": {2}, "verified": {3}}, - ChildMappings: []*core.DocumentMapping{nil, nil, { - IndexesByName: map[string][]int{"rating": {11}, "genre": {12}}, - }}, + IndexesByName: map[string][]int{ + "name": {authorNameInd}, + "age": {authorAgeInd}, + "published": {authorPublishedInd}, + "verified": {authorVerifiedInd}, + }, + ChildMappings: authorChildMappings, } } diff --git a/planner/mapper/targetable.go b/planner/mapper/targetable.go index bcfdb02ef8..0b571e6830 100644 --- a/planner/mapper/targetable.go +++ b/planner/mapper/targetable.go @@ -91,6 +91,17 @@ func (f *Filter) ToMap(mapping *core.DocumentMapping) map[string]any { return filterObjectToMap(mapping, f.Conditions) } +// HasIndex returns true if the filter has a condition that targets the +// a property with the given index. +func (f *Filter) HasIndex(index int) bool { + for k := range f.Conditions { + if propIndex, isOk := k.(*PropertyIndex); isOk && propIndex.Index == index { + return true + } + } + return false +} + func filterObjectToMap(mapping *core.DocumentMapping, obj map[connor.FilterKey]any) map[string]any { outmap := make(map[string]any) if obj == nil { diff --git a/planner/planner.go b/planner/planner.go index bcb0653633..7821b5aaaf 100644 --- a/planner/planner.go +++ b/planner/planner.go @@ -15,8 +15,10 @@ import ( "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/client/request" + "github.com/sourcenetwork/defradb/connor" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" + "github.com/sourcenetwork/defradb/planner/filter" "github.com/sourcenetwork/defradb/planner/mapper" ) @@ -296,16 +298,82 @@ func (p *Planner) expandMultiNode(multiNode MultiNode, parentPlan *selectTopNode return nil } +// expandTypeIndexJoinPlan does a plan graph expansion and other optimizations on typeIndexJoin. func (p *Planner) expandTypeIndexJoinPlan(plan *typeIndexJoin, parentPlan *selectTopNode) error { switch node := plan.joinPlan.(type) { case *typeJoinOne: - return p.expandPlan(node.subType, parentPlan) + return p.expandTypeJoin(&node.invertibleTypeJoin, parentPlan) case *typeJoinMany: - return p.expandPlan(node.subType, parentPlan) + return p.expandTypeJoin(&node.invertibleTypeJoin, parentPlan) } return client.NewErrUnhandledType("join plan", plan.joinPlan) } +func findFilteredByRelationFields( + conditions map[connor.FilterKey]any, + mapping *core.DocumentMapping, +) map[string]int { + filterProperties := filter.ExtractProperties(conditions) + filteredSubFields := make(map[string]int) + for _, prop := range filterProperties { + if childMapping := mapping.ChildMappings[prop.Index]; childMapping != nil { + if !prop.IsRelation() { + continue + } + for _, subProp := range prop.Fields { + for fieldName, indices := range childMapping.IndexesByName { + if indices[0] == subProp.Index { + filteredSubFields[fieldName] = subProp.Index + } + } + } + } + } + return filteredSubFields +} + +func (p *Planner) tryOptimizeJoinDirection(node *invertibleTypeJoin, parentPlan *selectTopNode) error { + filteredSubFields := findFilteredByRelationFields( + parentPlan.selectNode.filter.Conditions, + node.documentMapping, + ) + slct := node.subType.(*selectTopNode).selectNode + desc := slct.sourceInfo.collectionDescription + indexedFields := desc.CollectIndexedFields(&desc.Schema) + for _, indField := range indexedFields { + if ind, ok := filteredSubFields[indField.Name]; ok { + subInd := node.documentMapping.FirstIndexOfName(node.subTypeName) + relatedField := mapper.Field{Name: node.subTypeName, Index: subInd} + fieldFilter := filter.UnwrapRelation(filter.CopyField( + parentPlan.selectNode.filter, + relatedField, + mapper.Field{Name: indField.Name, Index: ind}, + ), relatedField) + err := node.invertJoinDirectionWithIndex(fieldFilter, indField) + if err != nil { + return err + } + break + } + } + + return nil +} + +// 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 { + return p.expandPlan(node.subType, parentPlan) + } + + err := p.tryOptimizeJoinDirection(node, parentPlan) + if err != nil { + return err + } + + return p.expandPlan(node.subType, parentPlan) +} + func (p *Planner) expandGroupNodePlan(topNodeSelect *selectTopNode) error { var sourceNode planNode var hasJoinNode bool @@ -406,9 +474,9 @@ func (p *Planner) walkAndReplacePlan(planNode, target, replace planNode) error { case *selectNode: node.source = replace case *typeJoinOne: - node.root = replace + node.replaceRoot(replace) case *typeJoinMany: - node.root = replace + node.replaceRoot(replace) case *pipeNode: /* Do nothing - pipe nodes should not be replaced */ // @todo: add more nodes that apply here diff --git a/planner/scan.go b/planner/scan.go index 256711b34e..f9a80705cb 100644 --- a/planner/scan.go +++ b/planner/scan.go @@ -11,12 +11,15 @@ package planner import ( + "github.com/sourcenetwork/immutable" + "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/db/base" "github.com/sourcenetwork/defradb/db/fetcher" "github.com/sourcenetwork/defradb/lens" + "github.com/sourcenetwork/defradb/planner/filter" "github.com/sourcenetwork/defradb/planner/mapper" "github.com/sourcenetwork/defradb/request/graphql/parser" ) @@ -90,7 +93,7 @@ func (n *scanNode) initFields(fields []mapper.Requestable) error { n.tryAddField(requestable.GetName()) // select might have its own select fields and filters fields case *mapper.Select: - n.tryAddField(requestable.Field.Name + "_id") // foreign key for type joins + n.tryAddField(requestable.Field.Name + request.RelatedObjectID) // foreign key for type joins err := n.initFields(requestable.Fields) if err != nil { return err @@ -133,6 +136,32 @@ func (n *scanNode) tryAddField(fieldName string) bool { return true } +func (scan *scanNode) initFetcher( + cid immutable.Option[string], + indexedField immutable.Option[client.FieldDescription], +) { + var f fetcher.Fetcher + if cid.HasValue() { + f = new(fetcher.VersionedFetcher) + } else { + f = new(fetcher.DocumentFetcher) + + if indexedField.HasValue() { + typeIndex := scan.documentMapping.FirstIndexOfName(indexedField.Value().Name) + field := mapper.Field{Index: typeIndex, Name: indexedField.Value().Name} + var indexFilter *mapper.Filter + scan.filter, indexFilter = filter.SplitByField(scan.filter, field) + if indexFilter != nil { + fieldDesc, _ := scan.desc.Schema.GetField(indexedField.Value().Name) + f = fetcher.NewIndexFetcher(f, fieldDesc, indexFilter) + } + } + + f = lens.NewFetcher(f, scan.p.db.LensRegistry()) + } + scan.fetcher = f +} + // Start starts the internal logic of the scanner // like the DocumentFetcher, and more. func (n *scanNode) Start() error { @@ -237,6 +266,7 @@ func (n *scanNode) executeExplain() map[string]any { "iterations": n.execInfo.iterations, "docFetches": n.execInfo.fetches.DocsFetched, "fieldFetches": n.execInfo.fetches.FieldsFetched, + "indexFetches": n.execInfo.fetches.IndexesFetched, } } @@ -258,26 +288,17 @@ func (n *scanNode) Explain(explainType request.ExplainType) (map[string]any, err // Merge implements mergeNode func (n *scanNode) Merge() bool { return true } -func (p *Planner) Scan(parsed *mapper.Select) (*scanNode, error) { - var f fetcher.Fetcher - if parsed.Cid.HasValue() { - f = new(fetcher.VersionedFetcher) - } else { - f = new(fetcher.DocumentFetcher) - f = lens.NewFetcher(f, p.db.LensRegistry()) - } +func (p *Planner) Scan( + mapperSelect *mapper.Select, + colDesc client.CollectionDescription, +) (*scanNode, error) { scan := &scanNode{ p: p, - fetcher: f, - slct: parsed, - docMapper: docMapper{parsed.DocumentMapping}, + slct: mapperSelect, + docMapper: docMapper{mapperSelect.DocumentMapping}, } - colDesc, err := p.getCollectionDesc(parsed.CollectionName) - if err != nil { - return nil, err - } - err = scan.initCollection(colDesc) + err := scan.initCollection(colDesc) if err != nil { return nil, err } @@ -294,8 +315,6 @@ func (p *Planner) Scan(parsed *mapper.Select) (*scanNode, error) { // we call Next() on the underlying scanNode only // once every 2 Next() calls on the multiScan type multiScanNode struct { - docMapper - scanNode *scanNode numReaders int numCalls int @@ -349,6 +368,10 @@ func (n *multiScanNode) Close() error { return n.scanNode.Close() } +func (n *multiScanNode) DocumentMap() *core.DocumentMapping { + return n.scanNode.DocumentMap() +} + func (n *multiScanNode) addReader() { n.numReaders++ } diff --git a/planner/select.go b/planner/select.go index 4fb9b143f2..21524ed31f 100644 --- a/planner/select.go +++ b/planner/select.go @@ -14,6 +14,7 @@ import ( cid "github.com/ipfs/go-cid" "github.com/sourcenetwork/immutable" + "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/db/base" @@ -250,10 +251,10 @@ func (n *selectNode) initSource() ([]aggregateNode, error) { // apply the root filter to the source // and rootSubType filters to the selectNode // @todo: simulate splitting for now - origScan, ok := n.source.(*scanNode) - if ok { - origScan.filter = n.filter + origScan, isScanNode := n.source.(*scanNode) + if isScanNode { origScan.showDeleted = n.selectReq.ShowDeleted + origScan.filter = n.filter n.filter = nil // If we have both a DocKey and a CID, then we need to run @@ -285,7 +286,31 @@ func (n *selectNode) initSource() ([]aggregateNode, error) { } } - return n.initFields(n.selectReq) + aggregates, err := n.initFields(n.selectReq) + if err != nil { + return nil, err + } + + if isScanNode { + origScan.initFetcher(n.selectReq.Cid, findFilteredByIndexedField(origScan)) + } + + return aggregates, nil +} + +func findFilteredByIndexedField(scanNode *scanNode) immutable.Option[client.FieldDescription] { + if scanNode.filter != nil { + indexedFields := scanNode.desc.CollectIndexedFields(&scanNode.desc.Schema) + for i := range indexedFields { + typeIndex := scanNode.documentMapping.FirstIndexOfName(indexedFields[i].Name) + if scanNode.filter.HasIndex(typeIndex) { + // we return the first found indexed field to keep it simple for now + // more sophisticated optimization logic can be added later + return immutable.Some(indexedFields[i]) + } + } + } + return immutable.None[client.FieldDescription]() } func (n *selectNode) initFields(selectReq *mapper.Select) ([]aggregateNode, error) { @@ -375,31 +400,6 @@ func (n *selectNode) addTypeIndexJoin(subSelect *mapper.Select) error { func (n *selectNode) Source() planNode { return n.source } -// func appendSource() {} - -// func (n *selectNode) initRender( -// fields []*client.FieldDescription, -// aliases []string, -//) error { -// return n.planner.render(fields, aliases) -// } - -// SubSelect is used for creating Select nodes used on sub selections, -// not to be used on the top level selection node. -// This allows us to disable rendering on all sub Select nodes -// and only run it at the end on the top level select node. -func (p *Planner) SubSelect(selectReq *mapper.Select) (planNode, error) { - plan, err := p.Select(selectReq) - if err != nil { - return nil, err - } - - // if this is a sub select plan, we need to remove the render node - // as the final top level selectTopNode will handle all sub renders - top := plan.(*selectTopNode) - return top, nil -} - func (p *Planner) SelectFromSource( selectReq *mapper.Select, source planNode, @@ -424,12 +424,12 @@ func (p *Planner) SelectFromSource( } if fromCollection { - desc, err := p.getCollectionDesc(selectReq.Name) + col, err := p.db.GetCollectionByName(p.ctx, selectReq.Name) if err != nil { return nil, err } - s.sourceInfo = sourceInfo{desc} + s.sourceInfo = sourceInfo{col.Description()} } aggregates, err := s.initFields(selectReq) diff --git a/planner/sum.go b/planner/sum.go index 0e1690898e..c5ef06a03a 100644 --- a/planner/sum.go +++ b/planner/sum.go @@ -77,12 +77,12 @@ func (p *Planner) isValueFloat( } if !source.ChildTarget.HasValue { - parentDescription, err := p.getCollectionDesc(parent.CollectionName) + parentCol, err := p.db.GetCollectionByName(p.ctx, parent.CollectionName) if err != nil { return false, err } - fieldDescription, fieldDescriptionFound := parentDescription.Schema.GetField(source.Name) + fieldDescription, fieldDescriptionFound := parentCol.Description().Schema.GetField(source.Name) if !fieldDescriptionFound { return false, client.NewErrFieldNotExist(source.Name) } @@ -125,12 +125,12 @@ func (p *Planner) isValueFloat( return false, nil } - childCollectionDescription, err := p.getCollectionDesc(child.CollectionName) + childCol, err := p.db.GetCollectionByName(p.ctx, child.CollectionName) if err != nil { return false, err } - fieldDescription, fieldDescriptionFound := childCollectionDescription.Schema.GetField(source.ChildTarget.Name) + fieldDescription, fieldDescriptionFound := childCol.Description().Schema.GetField(source.ChildTarget.Name) if !fieldDescriptionFound { return false, client.NewErrFieldNotExist(source.ChildTarget.Name) } diff --git a/planner/type_join.go b/planner/type_join.go index ee771b01fc..6e5d9a0d49 100644 --- a/planner/type_join.go +++ b/planner/type_join.go @@ -153,14 +153,14 @@ func (n *typeIndexJoin) simpleExplain() (map[string]any, error) { switch joinType := n.joinPlan.(type) { case *typeJoinOne: // Add the direction attribute. - if joinType.primary { - simpleExplainMap[joinDirectionLabel] = joinDirectionPrimaryLabel - } else { + if joinType.isSecondary { simpleExplainMap[joinDirectionLabel] = joinDirectionSecondaryLabel + } else { + simpleExplainMap[joinDirectionLabel] = joinDirectionPrimaryLabel } // Add the attribute(s). - simpleExplainMap[joinRootLabel] = joinType.subTypeFieldName + simpleExplainMap[joinRootLabel] = joinType.rootName simpleExplainMap[joinSubTypeNameLabel] = joinType.subTypeName subTypeExplainGraph, err := buildSimpleExplainGraph(joinType.subType) @@ -199,9 +199,24 @@ func (n *typeIndexJoin) Explain(explainType request.ExplainType) (map[string]any return n.simpleExplain() case request.ExecuteExplain: - return map[string]any{ + result := map[string]any{ "iterations": n.execInfo.iterations, - }, nil + } + var subScan *scanNode + if joinMany, isJoinMany := n.joinPlan.(*typeJoinMany); isJoinMany { + subScan = getScanNode(joinMany.subType) + } + if joinOne, isJoinOne := n.joinPlan.(*typeJoinOne); isJoinOne { + subScan = getScanNode(joinOne.subType) + } + if subScan != nil { + subScanExplain, err := subScan.Explain(explainType) + if err != nil { + return nil, err + } + result["subTypeScanNode"] = subScanExplain + } + return result, nil default: return nil, ErrUnknownExplainRequestType @@ -214,22 +229,7 @@ func (n *typeIndexJoin) Merge() bool { return true } // typeJoinOne is the plan node for a type index join // where the root type is the primary in a one-to-one relation request. type typeJoinOne struct { - documentIterator - docMapper - - p *Planner - - root planNode - subType planNode - - subTypeName string - subTypeFieldName string - - primary bool - secondaryFieldIndex immutable.Option[int] - - spans core.Spans - subSelect *mapper.Select + invertibleTypeJoin } func (p *Planner) makeTypeJoinOne( @@ -239,7 +239,7 @@ func (p *Planner) makeTypeJoinOne( ) (*typeJoinOne, error) { prepareScanNodeFilterForTypeJoin(parent, source, subType) - selectPlan, err := p.SubSelect(subType) + selectPlan, err := p.Select(subType) if err != nil { return nil, err } @@ -254,12 +254,12 @@ func (p *Planner) makeTypeJoinOne( // check if the field we're querying is the primary side of the relation isPrimary := subTypeFieldDesc.RelationType.IsSet(client.Relation_Type_Primary) - subTypeCollectionDesc, err := p.getCollectionDesc(subType.CollectionName) + subTypeCol, err := p.db.GetCollectionByName(p.ctx, subType.CollectionName) if err != nil { return nil, err } - subTypeField, subTypeFieldNameFound := subTypeCollectionDesc.GetFieldByRelation( + subTypeField, subTypeFieldNameFound := subTypeCol.Description().GetFieldByRelation( subTypeFieldDesc.RelationName, parent.sourceInfo.collectionDescription.Name, subTypeFieldDesc.Name, @@ -276,16 +276,26 @@ func (p *Planner) makeTypeJoinOne( ) } + dir := joinDirection{ + firstNode: source, + secondNode: selectPlan, + secondaryField: subTypeField.Name + request.RelatedObjectID, + primaryField: subTypeFieldDesc.Name + request.RelatedObjectID, + } + return &typeJoinOne{ - p: p, - root: source, - subSelect: subType, - subTypeName: subType.Name, - subTypeFieldName: subTypeField.Name, - subType: selectPlan, - primary: isPrimary, - secondaryFieldIndex: secondaryFieldIndex, - docMapper: docMapper{parent.documentMapping}, + invertibleTypeJoin: invertibleTypeJoin{ + docMapper: docMapper{parent.documentMapping}, + root: source, + subType: selectPlan, + subSelect: subType, + rootName: subTypeField.Name, + subTypeName: subType.Name, + isSecondary: !isPrimary, + secondaryFieldIndex: secondaryFieldIndex, + secondaryFetchLimit: 1, + dir: dir, + }, }, nil } @@ -293,139 +303,36 @@ func (n *typeJoinOne) Kind() string { return "typeJoinOne" } -func (n *typeJoinOne) Init() error { - if err := n.subType.Init(); err != nil { - return err - } - return n.root.Init() -} - -func (n *typeJoinOne) Start() error { - if err := n.subType.Start(); err != nil { - return err - } - return n.root.Start() -} - -func (n *typeJoinOne) Spans(spans core.Spans) { - n.root.Spans(spans) -} - -func (n *typeJoinOne) Next() (bool, error) { - hasNext, err := n.root.Next() - if err != nil || !hasNext { - return hasNext, err - } - - doc := n.root.Value() - if n.primary { - n.currentValue, err = n.valuesPrimary(doc) - } else { - n.currentValue, err = n.valuesSecondary(doc) - } +func fetchDocsWithFieldValue(plan planNode, fieldName string, val any, limit uint) ([]core.Doc, error) { + propIndex := plan.DocumentMap().FirstIndexOfName(fieldName) + setSubTypeFilterToScanNode(plan, propIndex, val) - if err != nil { - return false, err + if err := plan.Init(); err != nil { + return nil, NewErrSubTypeInit(err) } - return true, nil -} - -func (n *typeJoinOne) valuesSecondary(doc core.Doc) (core.Doc, error) { - propIndex := n.subType.DocumentMap().FirstIndexOfName(n.subTypeFieldName + request.RelatedObjectID) - // using the doc._key as a filter - setSubTypeFilterToScanNode(n.subType, propIndex, doc.GetKey()) - - // We have to reset the scan node after appending the new key-filter - if err := n.subType.Init(); err != nil { - return doc, NewErrSubTypeInit(err) - } - - next, err := n.subType.Next() - if !next || err != nil { - return doc, err - } - - subDoc := n.subType.Value() - doc.Fields[n.subSelect.Index] = subDoc - - if n.secondaryFieldIndex.HasValue() { - doc.Fields[n.secondaryFieldIndex.Value()] = subDoc.GetKey() - } - - return doc, nil -} - -func (n *typeJoinOne) valuesPrimary(doc core.Doc) (core.Doc, error) { - // get the subtype doc key - subDocKey := n.docMapper.documentMapping.FirstOfName(doc, n.subTypeName+request.RelatedObjectID) - - subDocKeyStr, ok := subDocKey.(string) - if !ok { - return doc, nil - } - - // create the collection key for the sub doc - slct := n.subType.(*selectTopNode).selectNode - desc := slct.sourceInfo.collectionDescription - subKeyIndexKey := base.MakeDocKey(desc, subDocKeyStr) - - // reset span - n.spans = core.NewSpans(core.NewSpan(subKeyIndexKey, subKeyIndexKey.PrefixEnd())) - - // do a point lookup with the new span (index key) - n.subType.Spans(n.spans) - - // re-initialize the sub type plan - if err := n.subType.Init(); err != nil { - return doc, NewErrSubTypeInit(err) - } - - // if we don't find any docs from our point span lookup - // or if we encounter an error just return the base doc, - // with an empty map for the subDoc - next, err := n.subType.Next() + docs := make([]core.Doc, 0, limit) + for { + next, err := plan.Next() + if err != nil { + return nil, err + } + if !next { + break + } - if err != nil { - return doc, err - } + docs = append(docs, plan.Value()) - if !next { - return doc, nil + if limit > 0 && len(docs) >= int(limit) { + break + } } - subDoc := n.subType.Value() - doc.Fields[n.subSelect.Index] = subDoc - - return doc, nil -} - -func (n *typeJoinOne) Close() error { - err := n.root.Close() - if err != nil { - return err - } - return n.subType.Close() + return docs, nil } -func (n *typeJoinOne) Source() planNode { return n.root } - type typeJoinMany struct { - documentIterator - docMapper - - p *Planner - - // the main type that is at the parent level of the request. - root planNode - rootName string - // the index to use to gather the subtype IDs - index *scanNode - // the subtype plan to get the subtype docs - subType planNode - subTypeName string - - subSelect *mapper.Select + invertibleTypeJoin } func prepareScanNodeFilterForTypeJoin( @@ -470,7 +377,7 @@ func (p *Planner) makeTypeJoinMany( ) (*typeJoinMany, error) { prepareScanNodeFilterForTypeJoin(parent, source, subType) - selectPlan, err := p.SubSelect(subType) + selectPlan, err := p.Select(subType) if err != nil { return nil, err } @@ -480,12 +387,12 @@ func (p *Planner) makeTypeJoinMany( return nil, client.NewErrFieldNotExist(subType.Name) } - subTypeCollectionDesc, err := p.getCollectionDesc(subType.CollectionName) + subTypeCol, err := p.db.GetCollectionByName(p.ctx, subType.CollectionName) if err != nil { return nil, err } - rootField, rootNameFound := subTypeCollectionDesc.GetFieldByRelation( + rootField, rootNameFound := subTypeCol.Description().GetFieldByRelation( subTypeFieldDesc.RelationName, parent.sourceInfo.collectionDescription.Name, subTypeFieldDesc.Name, @@ -495,14 +402,25 @@ func (p *Planner) makeTypeJoinMany( return nil, client.NewErrFieldNotExist(subTypeFieldDesc.RelationName) } + dir := joinDirection{ + firstNode: source, + secondNode: selectPlan, + secondaryField: rootField.Name + request.RelatedObjectID, + primaryField: subTypeFieldDesc.Name + request.RelatedObjectID, + } + return &typeJoinMany{ - p: p, - root: source, - subSelect: subType, - subTypeName: subType.Name, - rootName: rootField.Name, - subType: selectPlan, - docMapper: docMapper{parent.documentMapping}, + invertibleTypeJoin: invertibleTypeJoin{ + docMapper: docMapper{parent.documentMapping}, + root: source, + subType: selectPlan, + subSelect: subType, + rootName: rootField.Name, + isSecondary: true, + subTypeName: subType.Name, + secondaryFetchLimit: 0, + dir: dir, + }, }, nil } @@ -510,77 +428,194 @@ func (n *typeJoinMany) Kind() string { return "typeJoinMany" } -func (n *typeJoinMany) Init() error { - if err := n.subType.Init(); err != nil { - return err +func fetchPrimaryDoc(node, subNode planNode, parentProp string) (bool, error) { + subDoc := subNode.Value() + ind := subNode.DocumentMap().FirstIndexOfName(parentProp) + + docKeyStr, isStr := subDoc.Fields[ind].(string) + if !isStr { + return false, nil + } + + scan := getScanNode(node) + if scan == nil { + return false, nil + } + rootDocKey := base.MakeDocKey(scan.desc, docKeyStr) + + spans := core.NewSpans(core.NewSpan(rootDocKey, rootDocKey.PrefixEnd())) + + node.Spans(spans) + + if err := node.Init(); err != nil { + return false, NewErrSubTypeInit(err) + } + + hasValue, err := node.Next() + + if err != nil || !hasValue { + return false, err + } + + return true, nil +} + +type joinDirection struct { + firstNode planNode + secondNode planNode + secondaryField string + primaryField string + isInverted bool +} + +func (dir *joinDirection) invert() { + dir.isInverted = !dir.isInverted + dir.firstNode, dir.secondNode = dir.secondNode, dir.firstNode + dir.secondaryField, dir.primaryField = dir.primaryField, dir.secondaryField +} + +type invertibleTypeJoin struct { + documentIterator + docMapper + + root planNode + subType planNode + rootName string + subTypeName string + + subSelect *mapper.Select + + isSecondary bool + secondaryFieldIndex immutable.Option[int] + secondaryFetchLimit uint + + dir joinDirection +} + +func (join *invertibleTypeJoin) replaceRoot(node planNode) { + join.root = node + if join.dir.isInverted { + join.dir.secondNode = node + } else { + join.dir.firstNode = node } - return n.root.Init() } -func (n *typeJoinMany) Start() error { - if err := n.subType.Start(); err != nil { +func (join *invertibleTypeJoin) Init() error { + if err := join.subType.Init(); err != nil { return err } - return n.root.Start() + return join.root.Init() } -func (n *typeJoinMany) Spans(spans core.Spans) { - n.root.Spans(spans) +func (join *invertibleTypeJoin) Start() error { + if err := join.subType.Start(); err != nil { + return err + } + return join.root.Start() } -func (n *typeJoinMany) Next() (bool, error) { - hasNext, err := n.root.Next() - if err != nil || !hasNext { - return hasNext, err +func (join *invertibleTypeJoin) Close() error { + if err := join.root.Close(); err != nil { + return err } - n.currentValue = n.root.Value() + return join.subType.Close() +} + +func (join *invertibleTypeJoin) Spans(spans core.Spans) { + join.root.Spans(spans) +} + +func (join *invertibleTypeJoin) Source() planNode { return join.root } + +func (tj *invertibleTypeJoin) invert() { + tj.dir.invert() + tj.isSecondary = !tj.isSecondary +} - // check if theres an index - // if there is, scan and aggregate results - // if not, then manually scan the subtype table - subDocs := make([]core.Doc, 0) - if n.index != nil { - // @todo: handle index for one-to-many setup +func (join *invertibleTypeJoin) processSecondResult(secondDocs []core.Doc) (any, any) { + var secondResult any + var secondIDResult any + if join.secondaryFetchLimit == 1 { + if len(secondDocs) != 0 { + secondResult = secondDocs[0] + secondIDResult = secondDocs[0].GetKey() + } } else { - propIndex := n.subSelect.FirstIndexOfName(n.rootName + request.RelatedObjectID) - // using the doc._key as a filter - setSubTypeFilterToScanNode(n.subType, propIndex, n.currentValue.GetKey()) + secondResult = secondDocs + secondDocKeys := make([]string, len(secondDocs)) + for i, doc := range secondDocs { + secondDocKeys[i] = doc.GetKey() + } + secondIDResult = secondDocKeys + } + join.root.Value().Fields[join.subSelect.Index] = secondResult + if join.secondaryFieldIndex.HasValue() { + join.root.Value().Fields[join.secondaryFieldIndex.Value()] = secondIDResult + } + return secondResult, secondIDResult +} + +func (join *invertibleTypeJoin) Next() (bool, error) { + hasFirstValue, err := join.dir.firstNode.Next() + + if err != nil || !hasFirstValue { + return false, err + } - // reset scan node - if err := n.subType.Init(); err != nil { + firstDoc := join.dir.firstNode.Value() + + if join.isSecondary { + secondDocs, err := fetchDocsWithFieldValue( + join.dir.secondNode, + join.dir.secondaryField, + firstDoc.GetKey(), + join.secondaryFetchLimit, + ) + if err != nil { return false, err } - - for { - next, err := n.subType.Next() - if err != nil { - return false, err - } - if !next { - break + if join.dir.secondNode == join.root { + join.root.Value().Fields[join.subSelect.Index] = join.subType.Value() + } else { + secondResult, secondIDResult := join.processSecondResult(secondDocs) + join.dir.firstNode.Value().Fields[join.subSelect.Index] = secondResult + if join.secondaryFieldIndex.HasValue() { + join.dir.firstNode.Value().Fields[join.secondaryFieldIndex.Value()] = secondIDResult } + } + } else { + hasDoc, err := fetchPrimaryDoc(join.dir.secondNode, join.dir.firstNode, join.dir.primaryField) + if err != nil { + return false, err + } - subDoc := n.subType.Value() - subDocs = append(subDocs, subDoc) + if hasDoc { + join.root.Value().Fields[join.subSelect.Index] = join.subType.Value() } } - n.currentValue.Fields[n.subSelect.Index] = subDocs + join.currentValue = join.root.Value() + return true, nil } -func (n *typeJoinMany) Close() error { - if err := n.root.Close(); err != nil { - return err - } +func (join *invertibleTypeJoin) invertJoinDirectionWithIndex( + fieldFilter *mapper.Filter, + field client.FieldDescription, +) error { + subScan := getScanNode(join.subType) + subScan.tryAddField(join.rootName + request.RelatedObjectID) + subScan.filter = fieldFilter + subScan.initFetcher(immutable.Option[string]{}, immutable.Some(field)) - return n.subType.Close() -} + join.invert() -func (n *typeJoinMany) Source() planNode { return n.root } + return nil +} -func setSubTypeFilterToScanNode(plan planNode, propIndex int, key string) { +func setSubTypeFilterToScanNode(plan planNode, propIndex int, val any) { scan := getScanNode(plan) if scan == nil { return @@ -593,7 +628,7 @@ func setSubTypeFilterToScanNode(plan planNode, propIndex int, key string) { propertyIndex := &mapper.PropertyIndex{Index: propIndex} filterConditions := map[connor.FilterKey]any{ propertyIndex: map[connor.FilterKey]any{ - mapper.FilterEqOp: key, + mapper.FilterEqOp: val, }, } @@ -609,6 +644,11 @@ func getScanNode(plan planNode) *scanNode { return scanNode } node = node.Source() + if node == nil { + if topSelect, ok := plan.(*selectTopNode); ok { + node = topSelect.selectNode + } + } } return nil } diff --git a/tests/bench/bench_util.go b/tests/bench/bench_util.go index 4ffe998d88..184ca8c2ec 100644 --- a/tests/bench/bench_util.go +++ b/tests/bench/bench_util.go @@ -85,7 +85,7 @@ func ConstructSchema(fixture fixtures.Generator) (string, error) { // loop to get the schemas for i := 0; i < numTypes; i++ { - gql, err := fixtures.ExtractGQLFromType(fixture.Types()[i]) + gql, err := fixture.ExtractGQLFromType(fixture.Types()[i]) if err != nil { return "", errors.Wrap("failed generating GQL", err) } diff --git a/tests/bench/fixtures/fixtures.go b/tests/bench/fixtures/fixtures.go index 7b19b58f68..65ecf94e22 100644 --- a/tests/bench/fixtures/fixtures.go +++ b/tests/bench/fixtures/fixtures.go @@ -16,6 +16,7 @@ import ( "encoding/json" "fmt" "reflect" + "strings" "github.com/bxcodec/faker" @@ -28,19 +29,41 @@ var ( } ) +type Option func(*Generator) + +func OptionFieldDirective(typeName, field, directive string) Option { + return func(g *Generator) { + if g.directives == nil { + g.directives = make(map[string]map[string][]string) + } + if g.directives[typeName] == nil { + g.directives[typeName] = make(map[string][]string) + } + g.directives[typeName][field] = append(g.directives[typeName][field], directive) + } +} + type Generator struct { ctx context.Context schema string types []any + // map of type name to field name to list of directives + directives map[string]map[string][]string } -func ForSchema(ctx context.Context, schemaName string) Generator { - return Generator{ +func ForSchema(ctx context.Context, schemaName string, options ...Option) Generator { + g := Generator{ ctx: ctx, schema: schemaName, types: registeredFixtures[schemaName], } + + for _, o := range options { + o(&g) + } + + return g } // Types returns the defined types for this fixture set @@ -85,7 +108,7 @@ func (g Generator) GenerateDocs() ([]string, error) { // extractGQLFromType extracts a GraphQL SDL definition as a string // from a given type struct -func ExtractGQLFromType(t any) (string, error) { +func (g Generator) ExtractGQLFromType(t any) (string, error) { var buf bytes.Buffer if reflect.TypeOf(t).Kind() != reflect.Struct { @@ -104,7 +127,17 @@ func ExtractGQLFromType(t any) (string, error) { fname := f.Name ftype := f.Type.Name() gqlType := gTypeToGQLType[ftype] - fmt.Fprintf(&buf, "\t%s: %s\n", fname, gqlType) + + directives := "" + if g.directives != nil { + if dirsMap, ok := g.directives[name]; ok { + if dirs, ok := dirsMap[fname]; ok { + directives = " " + strings.Join(dirs, " ") + } + } + } + // write field's name, type and directives + fmt.Fprintf(&buf, "\t%s: %s%s\n", fname, gqlType, directives) } fmt.Fprint(&buf, "}") diff --git a/tests/bench/query/index/simple_test.go b/tests/bench/query/index/simple_test.go new file mode 100644 index 0000000000..e675086a2a --- /dev/null +++ b/tests/bench/query/index/simple_test.go @@ -0,0 +1,97 @@ +// Copyright 2023 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 query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/tests/bench/fixtures" + query "github.com/sourcenetwork/defradb/tests/bench/query/simple" +) + +var ( + userSimpleWithFilterQuery = ` + query { + User(filter: { Age: { _eq: 30 } }) { + _key + Name + Age + Points + Verified + } + } + ` +) + +func makeUserAgeIndexOption() fixtures.Option { + return fixtures.OptionFieldDirective("User", "Age", "@index") +} + +func Benchmark_Index_UserSimple_QueryWithFilterOnIndex_Sync_1(b *testing.B) { + ctx := context.Background() + err := query.RunQueryBenchGet( + b, + ctx, + fixtures.ForSchema(ctx, "user_simple", makeUserAgeIndexOption()), + 1, + userSimpleWithFilterQuery, + false, + ) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Index_UserSimple_QueryWithFilterOnIndex_Sync_10(b *testing.B) { + ctx := context.Background() + err := query.RunQueryBenchGet( + b, + ctx, + fixtures.ForSchema(ctx, "user_simple", makeUserAgeIndexOption()), + 10, + userSimpleWithFilterQuery, + false, + ) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Index_UserSimple_QueryWithFilterOnIndex_Sync_1000(b *testing.B) { + ctx := context.Background() + err := query.RunQueryBenchGet( + b, + ctx, + fixtures.ForSchema(ctx, "user_simple", makeUserAgeIndexOption()), + 1000, + userSimpleWithFilterQuery, + false, + ) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Index_UserSimple_QueryWithFilterOnIndex_Sync_10000(b *testing.B) { + ctx := context.Background() + err := query.RunQueryBenchGet( + b, + ctx, + fixtures.ForSchema(ctx, "user_simple", makeUserAgeIndexOption()), + 10000, + userSimpleWithFilterQuery, + false, + ) + if err != nil { + b.Fatal(err) + } +} diff --git a/tests/bench/query/simple/simple_test.go b/tests/bench/query/simple/simple_test.go index ca0d627275..a9791bcbc7 100644 --- a/tests/bench/query/simple/simple_test.go +++ b/tests/bench/query/simple/simple_test.go @@ -33,7 +33,7 @@ var ( func Benchmark_Query_UserSimple_Query_Sync_1(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -48,7 +48,7 @@ func Benchmark_Query_UserSimple_Query_Sync_1(b *testing.B) { func Benchmark_Query_UserSimple_Query_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -63,7 +63,7 @@ func Benchmark_Query_UserSimple_Query_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -78,7 +78,7 @@ func Benchmark_Query_UserSimple_Query_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/bench/query/simple/utils.go b/tests/bench/query/simple/utils.go index 6f95da50ed..8c6f82579b 100644 --- a/tests/bench/query/simple/utils.go +++ b/tests/bench/query/simple/utils.go @@ -27,7 +27,7 @@ var ( // log = logging.MustNewLogger("bench") ) -func runQueryBenchGet( +func RunQueryBenchGet( b *testing.B, ctx context.Context, fixture fixtures.Generator, diff --git a/tests/bench/query/simple/with_filter_test.go b/tests/bench/query/simple/with_filter_test.go index 86323e2beb..60081167a3 100644 --- a/tests/bench/query/simple/with_filter_test.go +++ b/tests/bench/query/simple/with_filter_test.go @@ -33,7 +33,7 @@ var ( func Benchmark_Query_UserSimple_Query_WithFilter_Sync_1(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -48,7 +48,7 @@ func Benchmark_Query_UserSimple_Query_WithFilter_Sync_1(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithFilter_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -63,7 +63,7 @@ func Benchmark_Query_UserSimple_Query_WithFilter_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithFilter_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -78,7 +78,7 @@ func Benchmark_Query_UserSimple_Query_WithFilter_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithFilter_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/bench/query/simple/with_limit_offset_test.go b/tests/bench/query/simple/with_limit_offset_test.go index 97dc523455..e47d8f347e 100644 --- a/tests/bench/query/simple/with_limit_offset_test.go +++ b/tests/bench/query/simple/with_limit_offset_test.go @@ -33,7 +33,7 @@ var ( func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_1(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -48,7 +48,7 @@ func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_1(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -63,7 +63,7 @@ func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -78,7 +78,7 @@ func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/bench/query/simple/with_multi_lookup_test.go b/tests/bench/query/simple/with_multi_lookup_test.go index 6af7b6e20a..2c744319a3 100644 --- a/tests/bench/query/simple/with_multi_lookup_test.go +++ b/tests/bench/query/simple/with_multi_lookup_test.go @@ -34,7 +34,7 @@ var ( func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -49,7 +49,7 @@ func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -64,7 +64,7 @@ func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/bench/query/simple/with_order_test.go b/tests/bench/query/simple/with_order_test.go index ec1a757f2b..2b12817713 100644 --- a/tests/bench/query/simple/with_order_test.go +++ b/tests/bench/query/simple/with_order_test.go @@ -33,7 +33,7 @@ var ( func Benchmark_Query_UserSimple_Query_WithSort_Sync_1(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -48,7 +48,7 @@ func Benchmark_Query_UserSimple_Query_WithSort_Sync_1(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSort_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -63,7 +63,7 @@ func Benchmark_Query_UserSimple_Query_WithSort_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSort_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -78,7 +78,7 @@ func Benchmark_Query_UserSimple_Query_WithSort_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSort_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/bench/query/simple/with_single_lookup_test.go b/tests/bench/query/simple/with_single_lookup_test.go index a2fb7e3b59..d432f730be 100644 --- a/tests/bench/query/simple/with_single_lookup_test.go +++ b/tests/bench/query/simple/with_single_lookup_test.go @@ -34,7 +34,7 @@ var ( func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_1(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -49,7 +49,7 @@ func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_1(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_10(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -64,7 +64,7 @@ func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_10(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_100(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), @@ -79,7 +79,7 @@ func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_100(b *testing.B) { func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_1000(b *testing.B) { ctx := context.Background() - err := runQueryBenchGet( + err := RunQueryBenchGet( b, ctx, fixtures.ForSchema(ctx, "user_simple"), diff --git a/tests/integration/client.go b/tests/integration/client.go index 06306724d6..a6159900cc 100644 --- a/tests/integration/client.go +++ b/tests/integration/client.go @@ -32,13 +32,13 @@ type ClientType string const ( // goClientType enables running the test suite using // the go implementation of the client.DB interface. - goClientType ClientType = "go" + GoClientType ClientType = "go" // httpClientType enables running the test suite using // the http implementation of the client.DB interface. - httpClientType ClientType = "http" + HTTPClientType ClientType = "http" // cliClientType enables running the test suite using // the cli implementation of the client.DB interface. - cliClientType ClientType = "cli" + CLIClientType ClientType = "cli" ) var ( @@ -65,13 +65,13 @@ func init() { // select the client implementation to use. func setupClient(s *state, node *net.Node) (impl clients.Client, err error) { switch s.clientType { - case httpClientType: + case HTTPClientType: impl, err = http.NewWrapper(node) - case cliClientType: + case CLIClientType: impl = cli.NewWrapper(node) - case goClientType: + case GoClientType: impl = node default: diff --git a/tests/integration/explain/execute/create_test.go b/tests/integration/explain/execute/create_test.go index e8ab75d48a..bd99ab39a4 100644 --- a/tests/integration/explain/execute/create_test.go +++ b/tests/integration/explain/execute/create_test.go @@ -48,6 +48,7 @@ func TestExecuteExplainMutationRequestWithCreate(t *testing.T) { "iterations": uint64(1), "docFetches": uint64(1), "fieldFetches": uint64(1), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/delete_test.go b/tests/integration/explain/execute/delete_test.go index 13411b5f5e..e924ce334c 100644 --- a/tests/integration/explain/execute/delete_test.go +++ b/tests/integration/explain/execute/delete_test.go @@ -51,6 +51,7 @@ func TestExecuteExplainMutationRequestWithDeleteUsingID(t *testing.T) { "iterations": uint64(2), "docFetches": uint64(1), "fieldFetches": uint64(1), + "indexFetches": uint64(0), }, }, }, @@ -99,6 +100,7 @@ func TestExecuteExplainMutationRequestWithDeleteUsingFilter(t *testing.T) { "iterations": uint64(2), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/group_test.go b/tests/integration/explain/execute/group_test.go index 3b7e42c845..9d4dc096f9 100644 --- a/tests/integration/explain/execute/group_test.go +++ b/tests/integration/explain/execute/group_test.go @@ -59,6 +59,7 @@ func TestExecuteExplainRequestWithGroup(t *testing.T) { "iterations": uint64(4), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/query_deleted_docs_test.go b/tests/integration/explain/execute/query_deleted_docs_test.go index 7642873b7f..cb1ebbcaa7 100644 --- a/tests/integration/explain/execute/query_deleted_docs_test.go +++ b/tests/integration/explain/execute/query_deleted_docs_test.go @@ -56,6 +56,7 @@ func TestExecuteExplainQueryDeletedDocs(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/scan_test.go b/tests/integration/explain/execute/scan_test.go index 85bd64229c..a68f175015 100644 --- a/tests/integration/explain/execute/scan_test.go +++ b/tests/integration/explain/execute/scan_test.go @@ -67,6 +67,7 @@ func TestExecuteExplainRequestWithAllDocumentsMatching(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, @@ -109,6 +110,7 @@ func TestExecuteExplainRequestWithNoDocuments(t *testing.T) { "iterations": uint64(1), "docFetches": uint64(0), "fieldFetches": uint64(0), + "indexFetches": uint64(0), }, }, }, @@ -172,6 +174,7 @@ func TestExecuteExplainRequestWithSomeDocumentsMatching(t *testing.T) { "iterations": uint64(2), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, @@ -235,6 +238,7 @@ func TestExecuteExplainRequestWithDocumentsButNoMatches(t *testing.T) { "iterations": uint64(1), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/top_level_test.go b/tests/integration/explain/execute/top_level_test.go index 6afa9cbfb2..360c9a3d2c 100644 --- a/tests/integration/explain/execute/top_level_test.go +++ b/tests/integration/explain/execute/top_level_test.go @@ -70,6 +70,7 @@ func TestExecuteExplainTopLevelAverageRequest(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, @@ -153,6 +154,7 @@ func TestExecuteExplainTopLevelCountRequest(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, @@ -227,6 +229,7 @@ func TestExecuteExplainTopLevelSumRequest(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/type_join_test.go b/tests/integration/explain/execute/type_join_test.go index 8e26f423bb..eb1e187485 100644 --- a/tests/integration/explain/execute/type_join_test.go +++ b/tests/integration/explain/execute/type_join_test.go @@ -56,6 +56,13 @@ func TestExecuteExplainRequestWithAOneToOneJoin(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(2), + "docFetches": uint64(2), + "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, @@ -115,6 +122,13 @@ func TestExecuteExplainWithMultipleOneToOneJoins(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(2), + "docFetches": uint64(2), + "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, @@ -125,6 +139,13 @@ func TestExecuteExplainWithMultipleOneToOneJoins(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(2), + "docFetches": uint64(2), + "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, @@ -187,6 +208,13 @@ func TestExecuteExplainWithTwoLevelDeepNestedJoins(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(2), + "docFetches": uint64(2), + "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/update_test.go b/tests/integration/explain/execute/update_test.go index d9469e4b4e..a1fa92b091 100644 --- a/tests/integration/explain/execute/update_test.go +++ b/tests/integration/explain/execute/update_test.go @@ -59,6 +59,7 @@ func TestExecuteExplainMutationRequestWithUpdateUsingIDs(t *testing.T) { "iterations": uint64(6), "docFetches": uint64(4), "fieldFetches": uint64(8), + "indexFetches": uint64(0), }, }, }, @@ -116,6 +117,7 @@ func TestExecuteExplainMutationRequestWithUpdateUsingFilter(t *testing.T) { "iterations": uint64(4), "docFetches": 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 a3070e8c42..9e906c475d 100644 --- a/tests/integration/explain/execute/with_average_test.go +++ b/tests/integration/explain/execute/with_average_test.go @@ -56,6 +56,7 @@ func TestExecuteExplainAverageRequestOnArrayField(t *testing.T) { "iterations": uint64(4), "docFetches": uint64(3), "fieldFetches": uint64(5), + "indexFetches": uint64(0), }, }, }, @@ -116,6 +117,13 @@ func TestExplainExplainAverageRequestOnJoinedField(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(12), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/with_count_test.go b/tests/integration/explain/execute/with_count_test.go index 236d0bf8af..4a30b9f52a 100644 --- a/tests/integration/explain/execute/with_count_test.go +++ b/tests/integration/explain/execute/with_count_test.go @@ -57,6 +57,13 @@ func TestExecuteExplainRequestWithCountOnOneToManyRelation(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(14), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/with_limit_test.go b/tests/integration/explain/execute/with_limit_test.go index 9a65ec1ec3..88a1666ca3 100644 --- a/tests/integration/explain/execute/with_limit_test.go +++ b/tests/integration/explain/execute/with_limit_test.go @@ -51,6 +51,7 @@ func TestExecuteExplainRequestWithBothLimitAndOffsetOnParent(t *testing.T) { "iterations": uint64(2), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), }, }, }, @@ -107,6 +108,13 @@ func TestExecuteExplainRequestWithBothLimitAndOffsetOnParentAndLimitOnChild(t *t "iterations": uint64(2), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(2), + "docFetches": uint64(4), + "fieldFetches": uint64(6), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/with_order_test.go b/tests/integration/explain/execute/with_order_test.go index d5b7ccfaed..9155523b20 100644 --- a/tests/integration/explain/execute/with_order_test.go +++ b/tests/integration/explain/execute/with_order_test.go @@ -52,6 +52,7 @@ func TestExecuteExplainRequestWithOrderFieldOnParent(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), }, }, }, @@ -135,6 +136,7 @@ func TestExecuteExplainRequestWithMultiOrderFieldsOnParent(t *testing.T) { "iterations": uint64(5), "docFetches": uint64(4), "fieldFetches": uint64(8), + "indexFetches": uint64(0), }, }, }, @@ -189,6 +191,13 @@ func TestExecuteExplainRequestWithOrderFieldOnChild(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(9), + "indexFetches": uint64(0), }, }, }, @@ -246,6 +255,13 @@ func TestExecuteExplainRequestWithOrderFieldOnBothParentAndChild(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(4), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(9), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain/execute/with_sum_test.go b/tests/integration/explain/execute/with_sum_test.go index c6df56c2e0..c37e3d0309 100644 --- a/tests/integration/explain/execute/with_sum_test.go +++ b/tests/integration/explain/execute/with_sum_test.go @@ -52,6 +52,7 @@ func TestExecuteExplainRequestWithSumOfInlineArrayField(t *testing.T) { "iterations": uint64(4), "docFetches": uint64(3), "fieldFetches": uint64(5), + "indexFetches": uint64(0), }, }, }, @@ -110,6 +111,13 @@ func TestExecuteExplainRequestSumOfRelatedOneToManyField(t *testing.T) { "iterations": uint64(3), "docFetches": uint64(2), "fieldFetches": uint64(2), + "indexFetches": uint64(0), + }, + "subTypeScanNode": dataMap{ + "iterations": uint64(5), + "docFetches": uint64(6), + "fieldFetches": uint64(9), + "indexFetches": uint64(0), }, }, }, diff --git a/tests/integration/explain_result_asserter.go b/tests/integration/explain_result_asserter.go new file mode 100644 index 0000000000..30126d4fe4 --- /dev/null +++ b/tests/integration/explain_result_asserter.go @@ -0,0 +1,162 @@ +// Copyright 2023 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 tests + +import ( + "encoding/json" + "fmt" + "testing" + + "github.com/sourcenetwork/immutable" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +const ( + iterationsProp = "iterations" + docFetchesProp = "docFetches" + fieldFetchesProp = "fieldFetches" + indexFetchesProp = "indexFetches" +) + +type dataMap = map[string]any + +// ExplainResultAsserter is a helper for asserting the result of an explain query. +// It allows asserting on a selected set of properties. +type ExplainResultAsserter struct { + iterations immutable.Option[int] + docFetches immutable.Option[int] + fieldFetches immutable.Option[int] + indexFetches immutable.Option[int] + filterMatches immutable.Option[int] + sizeOfResults immutable.Option[int] + planExecutions immutable.Option[uint64] +} + +func readNumberProp(t *testing.T, val any, prop string) uint64 { + switch v := val.(type) { + case uint64: + return v + case json.Number: + n, err := v.Int64() + require.NoError(t, err, fmt.Sprintf("Expected %s property to be a uint64", prop)) + return uint64(n) + default: + require.Fail(t, fmt.Sprintf("Unexpected type for %s property: %T", prop, val)) + } + return 0 +} + +func (a *ExplainResultAsserter) Assert(t *testing.T, result []dataMap) { + require.Len(t, result, 1, "Expected len(result) = 1, got %d", len(result)) + explainNode, ok := result[0]["explain"].(dataMap) + require.True(t, ok, "Expected explain none") + assert.Equal(t, explainNode["executionSuccess"], true, "Expected executionSuccess property") + if a.sizeOfResults.HasValue() { + actual := explainNode["sizeOfResult"] + assert.Equal(t, actual, a.sizeOfResults.Value(), + "Expected %d sizeOfResult, got %d", a.sizeOfResults.Value(), actual) + } + if a.planExecutions.HasValue() { + actual := explainNode["planExecutions"] + assert.Equal(t, actual, a.planExecutions.Value(), + "Expected %d planExecutions, got %d", a.planExecutions.Value(), actual) + } + selectTopNode, ok := explainNode["selectTopNode"].(dataMap) + require.True(t, ok, "Expected selectTopNode") + selectNode, ok := selectTopNode["selectNode"].(dataMap) + require.True(t, ok, "Expected selectNode") + + if a.filterMatches.HasValue() { + filterMatches, hasFilterMatches := selectNode["filterMatches"] + require.True(t, hasFilterMatches, "Expected filterMatches property") + assert.Equal(t, filterMatches, uint64(a.filterMatches.Value()), + "Expected %d filterMatches, got %d", a.filterMatches, filterMatches) + } + + scanNode, ok := selectNode["scanNode"].(dataMap) + subScanNode := map[string]any{} + if indexJoin, isJoin := selectNode["typeIndexJoin"].(dataMap); isJoin { + scanNode, ok = indexJoin["scanNode"].(dataMap) + subScanNode, _ = indexJoin["subTypeScanNode"].(dataMap) + } + require.True(t, ok, "Expected scanNode") + + getScanNodesProp := func(prop string) uint64 { + val, hasProp := scanNode[prop] + require.True(t, hasProp, fmt.Sprintf("Expected %s property", prop)) + actual := readNumberProp(t, val, prop) + if subScanNode[prop] != nil { + actual += readNumberProp(t, subScanNode[prop], "subTypeScanNode."+prop) + } + return actual + } + + if a.iterations.HasValue() { + actual := getScanNodesProp(iterationsProp) + assert.Equal(t, actual, uint64(a.iterations.Value()), + "Expected %d iterations, got %d", a.iterations.Value(), actual) + } + if a.docFetches.HasValue() { + actual := getScanNodesProp(docFetchesProp) + assert.Equal(t, actual, uint64(a.docFetches.Value()), + "Expected %d docFetches, got %d", a.docFetches.Value(), actual) + } + if a.fieldFetches.HasValue() { + actual := getScanNodesProp(fieldFetchesProp) + assert.Equal(t, actual, uint64(a.fieldFetches.Value()), + "Expected %d fieldFetches, got %d", a.fieldFetches.Value(), actual) + } + if a.indexFetches.HasValue() { + actual := getScanNodesProp(indexFetchesProp) + assert.Equal(t, actual, uint64(a.indexFetches.Value()), + "Expected %d indexFetches, got %d", a.indexFetches.Value(), actual) + } +} + +func (a *ExplainResultAsserter) WithIterations(iterations int) *ExplainResultAsserter { + a.iterations = immutable.Some[int](iterations) + return a +} + +func (a *ExplainResultAsserter) WithDocFetches(docFetches int) *ExplainResultAsserter { + a.docFetches = immutable.Some[int](docFetches) + return a +} + +func (a *ExplainResultAsserter) WithFieldFetches(fieldFetches int) *ExplainResultAsserter { + a.fieldFetches = immutable.Some[int](fieldFetches) + return a +} + +func (a *ExplainResultAsserter) WithIndexFetches(indexFetches int) *ExplainResultAsserter { + a.indexFetches = immutable.Some[int](indexFetches) + return a +} + +func (a *ExplainResultAsserter) WithFilterMatches(filterMatches int) *ExplainResultAsserter { + a.filterMatches = immutable.Some[int](filterMatches) + return a +} + +func (a *ExplainResultAsserter) WithSizeOfResults(sizeOfResults int) *ExplainResultAsserter { + a.sizeOfResults = immutable.Some[int](sizeOfResults) + return a +} + +func (a *ExplainResultAsserter) WithPlanExecutions(planExecutions uint64) *ExplainResultAsserter { + a.planExecutions = immutable.Some[uint64](planExecutions) + return a +} + +func NewExplainAsserter() *ExplainResultAsserter { + return &ExplainResultAsserter{} +} diff --git a/tests/integration/index/docs.go b/tests/integration/index/docs.go new file mode 100644 index 0000000000..505eadf98d --- /dev/null +++ b/tests/integration/index/docs.go @@ -0,0 +1,456 @@ +// Copyright 2023 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 index + +type docsCollection struct { + colName string + docs []map[string]any +} + +func getUserDocs() docsCollection { + return docsCollection{ + colName: "User", + docs: []map[string]any{ + { + "name": "Shahzad", + "age": 20, + "verified": false, + "email": "shahzad@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "iPhone Xs", + "year": 2022, + "type": "phone", + "specs": map[string]any{ + "CPU": 2.2, + "Chip": "Intel i3", + "RAM": 8, + "Storage": 512, + "OS": "iOS 12", + }, + }, + { + "model": "MacBook Pro", + "year": 2020, + "type": "laptop", + "specs": map[string]any{ + "CPU": 2.4, + "Chip": "Intel i5", + "RAM": 16, + "Storage": 2048, + "OS": "Yosemite", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 4635, + "city": "Montreal", + "country": "Canada", + "street": "Queen Mary Rd", + }, + }, + { + "name": "Bruno", + "age": 23, + "verified": true, + "email": "bruno@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{}, + }, + "address": map[string]any{ + "postalCode": 10001, + "city": "New York", + "country": "USA", + "street": "5th Ave", + }, + }, + { + "name": "Roy", + "age": 44, + "verified": true, + "email": "roy@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{}, + }, + "address": map[string]any{ + "postalCode": 90028, + "city": "Los Angeles", + "country": "USA", + "street": "Hollywood Blvd", + }, + }, + { + "name": "Fred", + "age": 28, + "verified": false, + "email": "fred@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "Samsung Galaxy S20", + "year": 2022, + "type": "phone", + "specs": map[string]any{ + "CPU": 2.0, + "Chip": "AMD Athlon", + "RAM": 8, + "Storage": 256, + "OS": "Android 11", + }, + }, + { + "model": "Lenovo ThinkPad", + "year": 2020, + "type": "laptop", + "specs": map[string]any{ + "CPU": 1.9, + "Chip": "AMD Ryzen", + "RAM": 8, + "Storage": 1024, + "OS": "Windows 10", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 6512, + "city": "Montreal", + "country": "Canada", + "street": "Park Ave", + }, + }, + { + "name": "John", + "age": 30, + "verified": false, + "email": "john@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "Google Pixel 5", + "year": 2022, + "type": "phone", + "specs": map[string]any{ + "CPU": 2.4, + "Chip": "Octa-core", + "RAM": 16, + "Storage": 512, + "OS": "Android 11", + }, + }, + { + "model": "Asus Vivobook", + "year": 2022, + "type": "laptop", + "specs": map[string]any{ + "CPU": 2.9, + "Chip": "Intel i7", + "RAM": 64, + "Storage": 2048, + "OS": "Windows 10", + }, + }, + { + "model": "Commodore 64", + "year": 1982, + "type": "computer", + "specs": map[string]any{ + "CPU": 0.1, + "Chip": "MOS 6510", + "RAM": 1, + "Storage": 1, + "OS": "Commodore BASIC 2.0", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 690, + "city": "Montreal", + "country": "Canada", + "street": "Notre-Dame St W", + }, + }, + { + "name": "Islam", + "age": 32, + "verified": false, + "email": "islam@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "iPhone 12s", + "year": 2018, + "type": "phone", + "specs": map[string]any{ + "CPU": 2.1, + "Chip": "A11 Bionic", + "RAM": 8, + "Storage": 1024, + "OS": "iOS 14", + }, + }, + { + "model": "MacBook Pro", + "year": 2023, + "type": "laptop", + "specs": map[string]any{ + "CPU": 2.6, + "Chip": "Apple M2 Max", + "RAM": 32, + "Storage": 1024, + "OS": "Sonoma 14", + }, + }, + { + "model": "iPad Pro", + "year": 2020, + "type": "tablet", + "specs": map[string]any{ + "CPU": 2.1, + "Chip": "Intel i5", + "RAM": 8, + "Storage": 512, + "OS": "iOS 14", + }, + }, + { + "model": "Playstation 5", + "year": 2022, + "type": "game_console", + "specs": map[string]any{ + "CPU": 3.5, + "Chip": "AMD Zen 2", + "RAM": 16, + "Storage": 825, + "OS": "FreeBSD", + }, + }, + { + "model": "Nokia 7610", + "year": 2003, + "type": "phone", + "specs": map[string]any{ + "CPU": 1.8, + "Chip": "Cortex A710", + "RAM": 12, + "Storage": 2, + "OS": "Symbian 7.0", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 80804, + "city": "Munich", + "country": "Germany", + "street": "Leopold Str", + }, + }, + { + "name": "Andy", + "age": 33, + "verified": true, + "email": "andy@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "Xiaomi Phone", + "year": 2022, + "type": "phone", + "specs": map[string]any{ + "CPU": 1.6, + "Chip": "AMD Octen", + "RAM": 8, + "Storage": 512, + "OS": "Android 11", + }, + }, + { + "model": "Alienware x16", + "year": 2018, + "type": "laptop", + "specs": map[string]any{ + "CPU": 3.2, + "Chip": "Intel i7", + "RAM": 64, + "Storage": 2048, + "OS": "Windows 9", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 101103, + "city": "London", + "country": "UK", + "street": "Baker St", + }, + }, + { + "name": "Addo", + "age": 42, + "verified": true, + "email": "addo@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "iPhone 10", + "year": 2021, + "type": "phone", + "specs": map[string]any{ + "CPU": 1.8, + "Chip": "Intel i3", + "RAM": 8, + "Storage": 256, + "OS": "iOS 12", + }, + }, + { + "model": "Acer Aspire 5", + "year": 2020, + "type": "laptop", + "specs": map[string]any{ + "CPU": 2.0, + "Chip": "Intel i5", + "RAM": 16, + "Storage": 512, + "OS": "Windows 10", + }, + }, + { + "model": "HyperX Headset", + "year": 2014, + "type": "headset", + "specs": map[string]any{ + "CPU": nil, + "Chip": nil, + "RAM": nil, + "Storage": nil, + "OS": nil, + }, + }, + { + "model": "Playstation 5", + "year": 2021, + "type": "game_console", + "specs": map[string]any{ + "CPU": 3.5, + "Chip": "AMD Zen 2", + "RAM": 16, + "Storage": 825, + "OS": "FreeBSD", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 403, + "city": "Ottawa", + "country": "Canada", + "street": "Bank St", + }, + }, + { + "name": "Keenan", + "age": 48, + "verified": true, + "email": "keenan@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "iPhone 13", + "year": 2022, + "type": "phone", + "specs": map[string]any{ + "CPU": 2.3, + "Chip": "M1", + "RAM": 8, + "Storage": 1024, + "OS": "iOS 14", + }, + }, + { + "model": "MacBook Pro", + "year": 2017, + "type": "laptop", + "specs": map[string]any{ + "CPU": 2.0, + "Chip": "A11 Bionic", + "RAM": 16, + "Storage": 512, + "OS": "Ventura", + }, + }, + { + "model": "iPad Mini", + "year": 2015, + "type": "tablet", + "specs": map[string]any{ + "CPU": 1.9, + "Chip": "Intel i3", + "RAM": 8, + "Storage": 1024, + "OS": "iOS 12", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 1600, + "city": "San Francisco", + "country": "USA", + "street": "Market St", + }, + }, + { + "name": "Chris", + "age": 55, + "verified": true, + "email": "chris@gmail.com", + "devices": docsCollection{ + colName: "Device", + docs: []map[string]any{ + { + "model": "Walkman", + "year": 2000, + "type": "phone", + "specs": map[string]any{ + "CPU": 1.8, + "Chip": "Cortex-A53 ", + "RAM": 8, + "Storage": 256, + "OS": "Android 11", + }, + }, + }, + }, + "address": map[string]any{ + "postalCode": 11680, + "city": "Toronto", + "country": "Canada", + "street": "Yonge St", + }, + }, + }, + } +} diff --git a/tests/integration/index/query_performance_test.go b/tests/integration/index/query_performance_test.go new file mode 100644 index 0000000000..eec8a13f4b --- /dev/null +++ b/tests/integration/index/query_performance_test.go @@ -0,0 +1,86 @@ +// Copyright 2023 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 index + +import ( + "fmt" + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func generateDocsForCollection(colIndex, count int) []any { + result := make([]any, 0, count) + for i := 0; i < count; i++ { + result = append(result, testUtils.CreateDoc{ + CollectionID: colIndex, + Doc: fmt.Sprintf(`{ + "name": "name-%d", + "age": %d, + "email": "email%d@gmail.com" + }`, i, i%100, i), + }) + } + return result +} + +func TestQueryPerformance_Simple(t *testing.T) { + const benchReps = 10 + const numDocs = 500 + + test1 := testUtils.TestCase{ + Actions: []any{ + testUtils.SchemaUpdate{Schema: ` + type User { + name: String + age: Int + email: String + } + `}, + testUtils.SchemaUpdate{ + Schema: ` + type IndexedUser { + name: String + age: Int @index + email: String + } + `, + }, + generateDocsForCollection(0, numDocs), + generateDocsForCollection(1, numDocs), + testUtils.Benchmark{ + Reps: benchReps, + BaseCase: testUtils.Request{Request: ` + query { + User(filter: {age: {_eq: 33}}) { + name + age + email + } + }`, + }, + OptimizedCase: testUtils.Request{Request: ` + query { + IndexedUser(filter: {age: {_eq: 33}}) { + name + age + email + } + }`, + }, + FocusClients: []testUtils.ClientType{testUtils.GoClientType}, + Factor: 5, + }, + }, + } + + testUtils.ExecuteTestCase(t, test1) +} diff --git a/tests/integration/index/query_with_index_combined_filter_test.go b/tests/integration/index/query_with_index_combined_filter_test.go new file mode 100644 index 0000000000..e5673d1ccf --- /dev/null +++ b/tests/integration/index/query_with_index_combined_filter_test.go @@ -0,0 +1,87 @@ +// Copyright 2023 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 index + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryWithIndex_IfIndexFilterWithRegular_ShouldFilter(t *testing.T) { + req := `query { + User(filter: { + name: {_in: ["Fred", "Islam", "Addo"]}, + age: {_gt: 40} + }) { + name + } + }` + test := testUtils.TestCase{ + Description: "Combination of a filter on regular and of an indexed field", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Addo"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(3).WithFieldFetches(6).WithIndexFetches(3), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_IfMultipleIndexFiltersWithRegular_ShouldFilter(t *testing.T) { + req := `query { + User(filter: { + name: {_like: "%a%"}, + age: {_gt: 30}, + email: {_like: "%m@gmail.com"} + }) { + name + } + }` + test := testUtils.TestCase{ + Description: "Combination of a filter on regular and of 2 indexed fields", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int @index + email: String + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Islam"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(6).WithFieldFetches(18), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/index/query_with_index_only_filter_test.go b/tests/integration/index/query_with_index_only_filter_test.go new file mode 100644 index 0000000000..f8e2bae6cc --- /dev/null +++ b/tests/integration/index/query_with_index_only_filter_test.go @@ -0,0 +1,534 @@ +// Copyright 2023 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 index + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryWithIndex_WithNonIndexedFields_ShouldFetchAllOfThem(t *testing.T) { + req := `query { + User(filter: {name: {_eq: "Islam"}}) { + name + age + } + }` + test := testUtils.TestCase{ + Description: "If there are non-indexed fields in the query, they should be fetched", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{{ + "name": "Islam", + "age": uint64(32), + }}, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(1).WithFieldFetches(2).WithIndexFetches(1), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithEqualFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {name: {_eq: "Islam"}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _eq filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Islam"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(1).WithFieldFetches(1).WithIndexFetches(1), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_IfSeveralDocsWithEqFilter_ShouldFetchAll(t *testing.T) { + req := `query { + User(filter: {name: {_eq: "Islam"}}) { + age + } + }` + test := testUtils.TestCase{ + Description: "If there are several docs matching _eq filter, they should be fetched", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.CreateDoc{ + CollectionID: 0, + Doc: `{ + "name": "Islam", + "age": 18 + }`, + }, + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"age": uint64(32)}, + {"age": uint64(18)}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(2), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithGreaterThanFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_gt: 48}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _gt filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Chris"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(1).WithFieldFetches(2).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithGreaterOrEqualFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_ge: 48}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _ge filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Keenan"}, + {"name": "Chris"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithLessThanFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_lt: 22}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _lt filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Shahzad"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(1).WithFieldFetches(2).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithLessOrEqualFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_le: 23}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _le filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Shahzad"}, + {"name": "Bruno"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNotEqualFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {name: {_ne: "Islam"}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _ne filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Roy"}, + {"name": "Addo"}, + {"name": "Andy"}, + {"name": "Fred"}, + {"name": "John"}, + {"name": "Bruno"}, + {"name": "Chris"}, + {"name": "Keenan"}, + {"name": "Shahzad"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(9).WithFieldFetches(9).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithInFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_in: [20, 33]}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _in filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Shahzad"}, + {"name": "Andy"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(2), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_IfSeveralDocsWithInFilter_ShouldFetchAll(t *testing.T) { + req := `query { + User(filter: {name: {_in: ["Islam"]}}) { + age + } + }` + test := testUtils.TestCase{ + Description: "If there are several docs matching _in filter, they should be fetched", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.CreateDoc{ + CollectionID: 0, + Doc: `{ + "name": "Islam", + "age": 18 + }`, + }, + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"age": uint64(32)}, + {"age": uint64(18)}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(2), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNotInFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {age: {_nin: [20, 23, 28, 33, 42, 55]}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _nin filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int @index + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "John"}, + {"name": "Islam"}, + {"name": "Roy"}, + {"name": "Keenan"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(4).WithFieldFetches(8).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithLikeFilter_ShouldFetch(t *testing.T) { + req1 := `query { + User(filter: {email: {_like: "a%"}}) { + name + } + }` + req2 := `query { + User(filter: {email: {_like: "%d@gmail.com"}}) { + name + } + }` + req3 := `query { + User(filter: {email: {_like: "%e%"}}) { + name + } + }` + req4 := `query { + User(filter: {email: {_like: "fred@gmail.com"}}) { + name + } + }` + req5 := `query { + User(filter: {email: {_like: "a%@gmail.com"}}) { + name + } + }` + req6 := `query { + User(filter: {email: {_like: "a%com%m"}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _like filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + email: String @index + } + `), + testUtils.Request{ + Request: req1, + Results: []map[string]any{ + {"name": "Addo"}, + {"name": "Andy"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + testUtils.Request{ + Request: req2, + Results: []map[string]any{ + {"name": "Fred"}, + {"name": "Shahzad"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + testUtils.Request{ + Request: req3, + Results: []map[string]any{ + {"name": "Fred"}, + {"name": "Keenan"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req3), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + testUtils.Request{ + Request: req4, + Results: []map[string]any{ + {"name": "Fred"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req4), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(1).WithFieldFetches(2).WithIndexFetches(10), + }, + testUtils.Request{ + Request: req5, + Results: []map[string]any{ + {"name": "Addo"}, + {"name": "Andy"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req5), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(4).WithIndexFetches(10), + }, + testUtils.Request{ + Request: req6, + Results: []map[string]any{}, + }, + testUtils.Request{ + Request: makeExplainQuery(req6), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(0).WithFieldFetches(0).WithIndexFetches(10), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndex_WithNotLikeFilter_ShouldFetch(t *testing.T) { + req := `query { + User(filter: {name: {_nlike: "%h%"}}) { + name + } + }` + test := testUtils.TestCase{ + Description: "Test index filtering with _nlike filter", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String @index + age: Int + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Roy"}, + {"name": "Addo"}, + {"name": "Andy"}, + {"name": "Fred"}, + {"name": "Bruno"}, + {"name": "Islam"}, + {"name": "Keenan"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(7).WithFieldFetches(7).WithIndexFetches(10), + }, + }, + } + + 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 new file mode 100644 index 0000000000..4a217e931c --- /dev/null +++ b/tests/integration/index/query_with_relation_filter_test.go @@ -0,0 +1,310 @@ +// Copyright 2023 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 index + +import ( + "testing" + + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +func TestQueryWithIndexOnOneToManyRelation_IfFilterOnIndexedRelation_ShouldFilter(t *testing.T) { + req1 := `query { + User(filter: { + devices: {model: {_eq: "MacBook Pro"}} + }) { + name + } + }` + 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{ + createSchemaWithDocs(` + type User { + name: String + age: Int + devices: [Device] + } + + type Device { + model: String @index + owner: User + } + `), + testUtils.Request{ + Request: req1, + Results: []map[string]any{ + {"name": "Islam"}, + {"name": "Shahzad"}, + {"name": "Keenan"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(6).WithFieldFetches(9).WithIndexFetches(3), + }, + testUtils.Request{ + Request: req2, + Results: []map[string]any{ + {"name": "Addo"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(3).WithIndexFetches(1), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndexOnOneToOnesSecondaryRelation_IfFilterOnIndexedRelation_ShouldFilter(t *testing.T) { + req1 := `query { + User(filter: { + address: {city: {_eq: "Munich"}} + }) { + name + } + }` + req2 := `query { + User(filter: { + address: {city: {_eq: "Montreal"}} + }) { + name + } + }` + test := testUtils.TestCase{ + Description: "Filter on indexed secondary relation field in 1-1 relation", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int + address: Address + } + + type Address { + user: User + city: String @index + } + `), + testUtils.Request{ + Request: req1, + Results: []map[string]any{ + {"name": "Islam"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(3).WithIndexFetches(1), + }, + testUtils.Request{ + Request: req2, + Results: []map[string]any{ + {"name": "Shahzad"}, + {"name": "Fred"}, + {"name": "John"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(6).WithFieldFetches(9).WithIndexFetches(3), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedFieldOfRelation_ShouldFilter(t *testing.T) { + req1 := `query { + User(filter: { + address: {city: {_eq: "London"}} + }) { + name + } + }` + req2 := `query { + User(filter: { + address: {city: {_eq: "Montreal"}} + }) { + name + } + }` + test := testUtils.TestCase{ + Description: "Filter on indexed field of primary relation in 1-1 relation", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int + address: Address @primary + } + + type Address { + user: User + city: String @index + street: String + } + `), + testUtils.Request{ + Request: req1, + Results: []map[string]any{ + {"name": "Andy"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(11).WithFieldFetches(12).WithIndexFetches(1), + }, + testUtils.Request{ + Request: req2, + Results: []map[string]any{ + {"name": "John"}, + {"name": "Fred"}, + {"name": "Shahzad"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(15).WithFieldFetches(18).WithIndexFetches(3), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndexOnOneToOnePrimaryRelation_IfFilterOnIndexedRelationWhileIndexedForeignField_ShouldFilter(t *testing.T) { + req := `query { + User(filter: { + address: {city: {_eq: "London"}} + }) { + name + } + }` + test := testUtils.TestCase{ + Description: "Filter on indexed field of primary relation while having indexed foreign field in 1-1 relation", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int + address: Address @primary @index + } + + type Address { + user: User + city: String @index + street: String + } + `), + testUtils.Request{ + Request: req, + Results: []map[string]any{ + {"name": "Andy"}, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(11).WithFieldFetches(12).WithIndexFetches(1), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} + +func TestQueryWithIndexOnOneToTwoRelation_IfFilterOnIndexedRelation_ShouldFilter(t *testing.T) { + req1 := `query { + User(filter: { + address: {city: {_eq: "Munich"}} + }) { + name + address { + city + } + } + }` + req2 := `query { + User(filter: { + devices: {model: {_eq: "Walkman"}} + }) { + name + devices { + model + } + } + }` + test := testUtils.TestCase{ + Description: "Filter on indexed relation field in 1-1 and 1-N relations", + Actions: []any{ + createSchemaWithDocs(` + type User { + name: String + age: Int + address: Address + devices: [Device] + } + + type Device { + model: String @index + owner: User + } + + type Address { + user: User + city: String @index + } + `), + testUtils.Request{ + Request: req1, + Results: []map[string]any{ + { + "name": "Islam", + "address": map[string]any{ + "city": "Munich", + }, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req1), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(3).WithIndexFetches(1), + }, + testUtils.Request{ + Request: req2, + Results: []map[string]any{ + { + "name": "Chris", + "devices": map[string]any{ + "model": "Walkman", + }, + }, + }, + }, + testUtils.Request{ + Request: makeExplainQuery(req2), + Asserter: testUtils.NewExplainAsserter().WithDocFetches(2).WithFieldFetches(3).WithIndexFetches(1), + }, + }, + } + + testUtils.ExecuteTestCase(t, test) +} diff --git a/tests/integration/index/utils.go b/tests/integration/index/utils.go new file mode 100644 index 0000000000..bb6cb89f14 --- /dev/null +++ b/tests/integration/index/utils.go @@ -0,0 +1,290 @@ +// Copyright 2023 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 index + +import ( + "fmt" + "strings" + + "github.com/sourcenetwork/immutable" + + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" + testUtils "github.com/sourcenetwork/defradb/tests/integration" +) + +// createSchemaWithDocs returns UpdateSchema action and CreateDoc actions +// with the documents that match the schema. +// The schema is parsed to get the list of properties, and the docs +// are created with the same properties. +// This allows us to have only one large list of docs with predefined +// properties, and create schemas with different properties from it. +func createSchemaWithDocs(schema string) []any { + userDocs := getUserDocs() + resultActions := make([]any, 0, len(userDocs.docs)+1) + resultActions = append(resultActions, testUtils.SchemaUpdate{Schema: schema}) + parser := schemaParser{} + typeDefs := parser.Parse(schema) + generator := createDocGenerator{types: typeDefs} + for _, doc := range userDocs.docs { + actions := generator.GenerateDocs(doc, userDocs.colName) + resultActions = append(resultActions, actions...) + } + return resultActions +} + +type createDocGenerator struct { + types map[string]typeDefinition +} + +func createDocJSON(doc map[string]any, typeDef *typeDefinition) string { + sb := strings.Builder{} + for propName := range doc { + format := `"%s": %v` + if _, isStr := doc[propName].(string); isStr { + format = `"%s": "%v"` + } + if sb.Len() == 0 { + sb.WriteString("{\n") + } else { + sb.WriteString(",\n") + } + sb.WriteString(fmt.Sprintf(format, propName, doc[propName])) + } + sb.WriteString("\n}") + return sb.String() +} + +func toRequestedDoc(doc map[string]any, typeDef *typeDefinition) map[string]any { + result := make(map[string]any) + for _, prop := range typeDef.props { + if prop.isRelation { + continue + } + result[prop.name] = doc[prop.name] + } + for name, val := range doc { + if strings.HasSuffix(name, request.RelatedObjectID) { + result[name] = val + } + } + return result +} + +func (this *createDocGenerator) generatePrimary( + doc map[string]any, + typeDef *typeDefinition, +) (map[string]any, []any) { + result := []any{} + requested := toRequestedDoc(doc, typeDef) + for _, prop := range typeDef.props { + if prop.isRelation { + if _, hasProp := doc[prop.name]; hasProp { + if prop.isPrimary.Value() { + subType := this.types[prop.typeStr] + subDoc := toRequestedDoc(doc[prop.name].(map[string]any), &subType) + jsonSubDoc := createDocJSON(subDoc, &subType) + clientSubDoc, err := client.NewDocFromJSON([]byte(jsonSubDoc)) + if err != nil { + panic("Failed to create doc from JSON: " + err.Error()) + } + requested[prop.name+request.RelatedObjectID] = clientSubDoc.Key().String() + result = append(result, testUtils.CreateDoc{CollectionID: subType.index, Doc: jsonSubDoc}) + } + } + } + } + return requested, result +} + +func (this *createDocGenerator) GenerateDocs(doc map[string]any, typeName string) []any { + typeDef := this.types[typeName] + + requested, result := this.generatePrimary(doc, &typeDef) + docStr := createDocJSON(requested, &typeDef) + + result = append(result, testUtils.CreateDoc{CollectionID: typeDef.index, Doc: docStr}) + + var docKey string + for _, prop := range typeDef.props { + if prop.isRelation { + if _, hasProp := doc[prop.name]; hasProp { + if !prop.isPrimary.Value() { + if docKey == "" { + clientDoc, err := client.NewDocFromJSON([]byte(docStr)) + if err != nil { + panic("Failed to create doc from JSON: " + err.Error()) + } + docKey = clientDoc.Key().String() + } + actions := this.generateSecondaryDocs(doc, typeName, &prop, docKey) + result = append(result, actions...) + } + } + } + } + return result +} + +func (this *createDocGenerator) generateSecondaryDocs( + primaryDoc map[string]any, + primaryTypeName string, + relProp *propDefinition, + primaryDocKey string, +) []any { + result := []any{} + relTypeDef := this.types[relProp.typeStr] + primaryPropName := "" + for _, relDocProp := range relTypeDef.props { + if relDocProp.typeStr == primaryTypeName && relDocProp.isPrimary.Value() { + primaryPropName = relDocProp.name + request.RelatedObjectID + switch relVal := primaryDoc[relProp.name].(type) { + case docsCollection: + for _, relDoc := range relVal.docs { + relDoc[primaryPropName] = primaryDocKey + actions := this.GenerateDocs(relDoc, relTypeDef.name) + result = append(result, actions...) + } + case map[string]any: + relVal[primaryPropName] = primaryDocKey + actions := this.GenerateDocs(relVal, relTypeDef.name) + result = append(result, actions...) + } + } + } + return result +} + +type propDefinition struct { + name string + typeStr string + isArray bool + isRelation bool + isPrimary immutable.Option[bool] +} + +type typeDefinition struct { + name string + index int + props map[string]propDefinition +} + +type schemaParser struct { + types map[string]typeDefinition + schemaLines []string + firstRelationType string + currentTypeDef typeDefinition + relationTypesMap map[string]map[string]string +} + +func (p *schemaParser) Parse(schema string) map[string]typeDefinition { + p.types = make(map[string]typeDefinition) + p.relationTypesMap = make(map[string]map[string]string) + p.schemaLines = strings.Split(schema, "\n") + p.findTypes() + + for _, line := range p.schemaLines { + line = strings.TrimSpace(line) + if strings.HasPrefix(line, "type ") { + typeNameEndPos := strings.Index(line[5:], " ") + typeName := strings.TrimSpace(line[5 : 5+typeNameEndPos]) + p.currentTypeDef = p.types[typeName] + continue + } + if strings.HasPrefix(line, "}") { + p.types[p.currentTypeDef.name] = p.currentTypeDef + continue + } + pos := strings.Index(line, ":") + if pos != -1 { + p.defineProp(line, pos) + } + } + p.resolvePrimaryRelations() + return p.types +} + +func (p *schemaParser) findTypes() { + typeIndex := 0 + for _, line := range p.schemaLines { + line = strings.TrimSpace(line) + if strings.HasPrefix(line, "type ") { + typeNameEndPos := strings.Index(line[5:], " ") + typeName := strings.TrimSpace(line[5 : 5+typeNameEndPos]) + p.types[typeName] = typeDefinition{name: typeName, index: typeIndex, props: make(map[string]propDefinition)} + typeIndex++ + } + } +} + +func (p *schemaParser) defineProp(line string, pos int) { + prop := propDefinition{name: line[:pos]} + prop.typeStr = strings.TrimSpace(line[pos+1:]) + typeEndPos := strings.Index(prop.typeStr, " ") + if typeEndPos != -1 { + prop.typeStr = prop.typeStr[:typeEndPos] + } + if prop.typeStr[0] == '[' { + prop.isArray = true + prop.typeStr = prop.typeStr[1 : len(prop.typeStr)-1] + } + if _, isRelation := p.types[prop.typeStr]; isRelation { + prop.isRelation = true + if prop.isArray { + prop.isPrimary = immutable.Some(false) + } else if strings.Contains(line[pos+len(prop.typeStr)+2:], "@primary") { + prop.isPrimary = immutable.Some(true) + } + relMap := p.relationTypesMap[prop.typeStr] + if relMap == nil { + relMap = make(map[string]string) + } + relMap[prop.name] = p.currentTypeDef.name + p.relationTypesMap[prop.typeStr] = relMap + if p.firstRelationType == "" { + p.firstRelationType = p.currentTypeDef.name + } + } + p.currentTypeDef.props[prop.name] = prop +} + +func (p *schemaParser) resolvePrimaryRelations() { + for typeName, relationProps := range p.relationTypesMap { + typeDef := p.types[typeName] + for _, prop := range typeDef.props { + for relPropName, relPropType := range relationProps { + if prop.typeStr == relPropType { + relatedTypeDef := p.types[relPropType] + relatedProp := relatedTypeDef.props[relPropName] + if !relatedProp.isPrimary.HasValue() { + relatedProp.isPrimary = immutable.Some(typeName == p.firstRelationType) + relatedTypeDef.props[relPropName] = relatedProp + p.types[relPropType] = relatedTypeDef + delete(p.relationTypesMap, relPropType) + } + if !prop.isPrimary.HasValue() { + val := typeName != p.firstRelationType + if relatedProp.isPrimary.HasValue() { + val = !relatedProp.isPrimary.Value() + } + prop.isPrimary = immutable.Some(val) + typeDef.props[prop.name] = prop + } + } + } + } + p.types[typeName] = typeDef + } +} + +func makeExplainQuery(req string) string { + return "query @explain(type: execute) " + req[6:] +} diff --git a/tests/integration/results.go b/tests/integration/results.go index 176b2e4cf2..35a2249c0b 100644 --- a/tests/integration/results.go +++ b/tests/integration/results.go @@ -28,7 +28,7 @@ type AnyOf []any // The comparison is relaxed when using client types other than goClientType. func assertResultsAnyOf(t *testing.T, client ClientType, expected AnyOf, actual any, msgAndArgs ...any) { switch client { - case httpClientType, cliClientType: + case HTTPClientType, CLIClientType: if !areResultsAnyOf(expected, actual) { assert.Contains(t, expected, actual, msgAndArgs...) } @@ -42,7 +42,7 @@ func assertResultsAnyOf(t *testing.T, client ClientType, expected AnyOf, actual // The comparison is relaxed when using client types other than goClientType. func assertResultsEqual(t *testing.T, client ClientType, expected any, actual any, msgAndArgs ...any) { switch client { - case httpClientType, cliClientType: + case HTTPClientType, CLIClientType: if !areResultsEqual(expected, actual) { assert.EqualValues(t, expected, actual, msgAndArgs...) } diff --git a/tests/integration/state.go b/tests/integration/state.go index a60c867565..ca795a2492 100644 --- a/tests/integration/state.go +++ b/tests/integration/state.go @@ -84,6 +84,9 @@ type state struct { // Indexes, by index, by collection index, by node index. indexes [][][]client.IndexDescription + + // isBench indicates wether the test is currently being benchmarked. + isBench bool } // newState returns a new fresh state for the given testCase. diff --git a/tests/integration/test_case.go b/tests/integration/test_case.go index 10f3cf7262..fabdccbbfd 100644 --- a/tests/integration/test_case.go +++ b/tests/integration/test_case.go @@ -11,6 +11,8 @@ package tests import ( + "testing" + "github.com/sourcenetwork/immutable" "github.com/sourcenetwork/defradb/client" @@ -254,6 +256,36 @@ type GetIndexes struct { ExpectedError string } +// ResultAsserter is an interface that can be implemented to provide custom result +// assertions. +type ResultAsserter interface { + // Assert will be called with the test and the result of the request. + Assert(t *testing.T, result []map[string]any) +} + +// ResultAsserterFunc is a function that can be used to implement the ResultAsserter +type ResultAsserterFunc func(*testing.T, []map[string]any) (bool, string) + +func (f ResultAsserterFunc) Assert(t *testing.T, result []map[string]any) { + f(t, result) +} + +// Benchmark is an action that will run another test action for benchmark test. +// It will run benchmarks for a base case and optimized case and assert that +// the optimized case performs better by at least the given factor. +type Benchmark struct { + // BaseCase is a test action which is the base case to benchmark. + BaseCase any + // OptimizedCase is a test action which is the optimized case to benchmark. + OptimizedCase any + // Reps is the number of times to run the benchmark. + Reps int + // FocusClients is the list of clients to run the benchmark on. + FocusClients []ClientType + // Factor is the factor by which the optimized case should be better than the base case. + Factor float64 +} + // Request represents a standard Defra (GQL) request. type Request struct { // NodeID may hold the ID (index) of a node to execute this request on. @@ -271,6 +303,9 @@ type Request struct { // The expected (data) results of the issued request. Results []map[string]any + // Asserter is an optional custom result asserter. + Asserter ResultAsserter + // Any error expected from the action. Optional. // // String can be a partial, and the test will pass if an error is returned that diff --git a/tests/integration/utils2.go b/tests/integration/utils2.go index e61cb5abce..01c6c7c69f 100644 --- a/tests/integration/utils2.go +++ b/tests/integration/utils2.go @@ -120,19 +120,20 @@ func ExecuteTestCase( t *testing.T, testCase TestCase, ) { + flattenActions(&testCase) collectionNames := getCollectionNames(testCase) changeDetector.PreTestChecks(t, collectionNames) skipIfMutationTypeUnsupported(t, testCase.SupportedMutationTypes) var clients []ClientType if httpClient { - clients = append(clients, httpClientType) + clients = append(clients, HTTPClientType) } if goClient { - clients = append(clients, goClientType) + clients = append(clients, GoClientType) } if cliClient { - clients = append(clients, cliClientType) + clients = append(clients, CLIClientType) } var databases []DatabaseType @@ -170,12 +171,8 @@ func executeTestCase( log.Info( ctx, testCase.Description, - logging.NewKV("badgerFile", badgerFile), - logging.NewKV("badgerInMemory", badgerInMemory), - logging.NewKV("inMemoryStore", inMemoryStore), - logging.NewKV("httpClient", httpClient), - logging.NewKV("goClient", goClient), - logging.NewKV("cliClient", cliClient), + logging.NewKV("database", dbt), + logging.NewKV("client", clientType), logging.NewKV("mutationType", mutationType), logging.NewKV("databaseDir", databaseDir), logging.NewKV("changeDetector.Enabled", changeDetector.Enabled), @@ -185,7 +182,6 @@ func executeTestCase( logging.NewKV("changeDetector.Repository", changeDetector.Repository), ) - flattenActions(&testCase) startActionIndex, endActionIndex := getActionRange(testCase) s := newState(ctx, t, testCase, dbt, clientType, collectionNames) @@ -203,113 +199,166 @@ func executeTestCase( refreshIndexes(s) for i := startActionIndex; i <= endActionIndex; i++ { - switch action := testCase.Actions[i].(type) { - case ConfigureNode: - configureNode(s, action) + performAction(s, i, testCase.Actions[i]) + } - case Restart: - restartNodes(s, i) + // Notify any active subscriptions that all requests have been sent. + close(s.allActionsDone) - case ConnectPeers: - connectPeers(s, action) + for _, resultsChan := range s.subscriptionResultsChans { + select { + case subscriptionAssert := <-resultsChan: + // We want to assert back in the main thread so failures get recorded properly + subscriptionAssert() - case ConfigureReplicator: - configureReplicator(s, action) + // a safety in case the stream hangs - we don't want the tests to run forever. + case <-time.After(subscriptionTimeout): + assert.Fail(t, "timeout occurred while waiting for data stream", testCase.Description) + } + } +} - case DeleteReplicator: - deleteReplicator(s, action) +func performAction( + s *state, + actionIndex int, + act any, +) { + switch action := act.(type) { + case ConfigureNode: + configureNode(s, action) - case SubscribeToCollection: - subscribeToCollection(s, action) + case Restart: + restartNodes(s, actionIndex) - case UnsubscribeToCollection: - unsubscribeToCollection(s, action) + case ConnectPeers: + connectPeers(s, action) - case GetAllP2PCollections: - getAllP2PCollections(s, action) + case ConfigureReplicator: + configureReplicator(s, action) - case SchemaUpdate: - updateSchema(s, action) + case DeleteReplicator: + deleteReplicator(s, action) - case SchemaPatch: - patchSchema(s, action) + case SubscribeToCollection: + subscribeToCollection(s, action) - case SetDefaultSchemaVersion: - setDefaultSchemaVersion(s, action) + case UnsubscribeToCollection: + unsubscribeToCollection(s, action) - case ConfigureMigration: - configureMigration(s, action) + case GetAllP2PCollections: + getAllP2PCollections(s, action) - case GetMigrations: - getMigrations(s, action) + case SchemaUpdate: + updateSchema(s, action) - case CreateDoc: - createDoc(s, action) + case SchemaPatch: + patchSchema(s, action) - case DeleteDoc: - deleteDoc(s, action) + case SetDefaultSchemaVersion: + setDefaultSchemaVersion(s, action) - case UpdateDoc: - updateDoc(s, action) + case ConfigureMigration: + configureMigration(s, action) - case CreateIndex: - createIndex(s, action) + case GetMigrations: + getMigrations(s, action) - case DropIndex: - dropIndex(s, action) + case CreateDoc: + createDoc(s, action) - case GetIndexes: - getIndexes(s, action) + case DeleteDoc: + deleteDoc(s, action) - case BackupExport: - backupExport(s, action) + case UpdateDoc: + updateDoc(s, action) - case BackupImport: - backupImport(s, action) + case CreateIndex: + createIndex(s, action) - case TransactionCommit: - commitTransaction(s, action) + case DropIndex: + dropIndex(s, action) - case SubscriptionRequest: - executeSubscriptionRequest(s, action) + case GetIndexes: + getIndexes(s, action) - case Request: - executeRequest(s, action) + case BackupExport: + backupExport(s, action) - case ExplainRequest: - executeExplainRequest(s, action) + case BackupImport: + backupImport(s, action) - case IntrospectionRequest: - assertIntrospectionResults(s, action) + case TransactionCommit: + commitTransaction(s, action) - case ClientIntrospectionRequest: - assertClientIntrospectionResults(s, action) + case SubscriptionRequest: + executeSubscriptionRequest(s, action) - case WaitForSync: - waitForSync(s, action) + case Request: + executeRequest(s, action) - case SetupComplete: - // no-op, just continue. + case ExplainRequest: + executeExplainRequest(s, action) - default: - t.Fatalf("Unknown action type %T", action) - } - } + case IntrospectionRequest: + assertIntrospectionResults(s, action) - // Notify any active subscriptions that all requests have been sent. - close(s.allActionsDone) + case ClientIntrospectionRequest: + assertClientIntrospectionResults(s, action) - for _, resultsChan := range s.subscriptionResultsChans { - select { - case subscriptionAssert := <-resultsChan: - // We want to assert back in the main thread so failures get recorded properly - subscriptionAssert() + case WaitForSync: + waitForSync(s, action) - // a safety in case the stream hangs - we don't want the tests to run forever. - case <-time.After(subscriptionTimeout): - assert.Fail(t, "timeout occurred while waiting for data stream", testCase.Description) + case Benchmark: + benchmarkAction(s, actionIndex, action) + + case SetupComplete: + // no-op, just continue. + + default: + s.t.Fatalf("Unknown action type %T", action) + } +} + +func benchmarkAction( + s *state, + actionIndex int, + bench Benchmark, +) { + if s.dbt == defraIMType { + // Benchmarking makes no sense for test in-memory storage + return + } + if len(bench.FocusClients) > 0 { + isFound := false + for _, clientType := range bench.FocusClients { + if s.clientType == clientType { + isFound = true + break + } + } + if !isFound { + return } } + + runBench := func(benchCase any) time.Duration { + startTime := time.Now() + for i := 0; i < bench.Reps; i++ { + performAction(s, actionIndex, benchCase) + } + return time.Since(startTime) + } + + s.isBench = true + defer func() { s.isBench = false }() + + baseElapsedTime := runBench(bench.BaseCase) + optimizedElapsedTime := runBench(bench.OptimizedCase) + + factoredBaseTime := int64(float64(baseElapsedTime) / bench.Factor) + assert.Greater(s.t, factoredBaseTime, optimizedElapsedTime, + "Optimized case should be faster at least by factor of %.2f than the base case. Base: %d, Optimized: %d (μs)", + bench.Factor, optimizedElapsedTime.Microseconds(), baseElapsedTime.Microseconds()) } // getCollectionNames gets an ordered, unique set of collection names across all nodes @@ -1369,6 +1418,7 @@ func executeRequest( &result.GQL, action.Results, action.ExpectedError, + action.Asserter, nodeID, anyOfByFieldKey, ) @@ -1434,6 +1484,7 @@ func executeSubscriptionRequest( finalResult, action.Results, action.ExpectedError, + nil, // anyof is not yet supported by subscription requests 0, map[docFieldKey][]any{}, @@ -1506,10 +1557,12 @@ func assertRequestResults( result *client.GQLResult, expectedResults []map[string]any, expectedError string, + asserter ResultAsserter, nodeID int, anyOfByField map[docFieldKey][]any, ) bool { - if AssertErrors(s.t, s.testCase.Description, result.Errors, expectedError) { + // we skip assertion benchmark because you don't specify expected result for benchmark. + if AssertErrors(s.t, s.testCase.Description, result.Errors, expectedError) || s.isBench { return true } @@ -1520,9 +1573,16 @@ func assertRequestResults( // Note: if result.Data == nil this panics (the panic seems useful while testing). resultantData := result.Data.([]map[string]any) + if asserter != nil { + asserter.Assert(s.t, resultantData) + return true + } + log.Info(s.ctx, "", logging.NewKV("RequestResults", result.Data)) - require.Equal(s.t, len(expectedResults), len(resultantData), s.testCase.Description) + // compare results + require.Equal(s.t, len(expectedResults), len(resultantData), + s.testCase.Description+" \n(number of results don't match)") for docIndex, result := range resultantData { expectedResult := expectedResults[docIndex]