From 6ccae7f36eb4bd770110a3e472bb26a0096a3a5b Mon Sep 17 00:00:00 2001 From: AndrewSisley Date: Fri, 21 Oct 2022 10:30:27 -0400 Subject: [PATCH] refactor: Cleanup parsing logic (#909) * Remove unused QueryType prop * Remove unused order.statement prop * Replace optionalDocKeys with client.Option Type was added before we had client.Option * Remove legacy commit entry Should have been removed in the PR that removed the commit query * Remove unused statement property * Remove unused func from interface * Remove Root from Field * Remove unused func * Parse aggregates in parser package * Remove statement from parser.Select * Remove ast from OperationDefinition * Remove unused Name from OperationDefinition * Make parser.Alias optional The parser models will shortly be exposed as part of the public interface, and it should not be on the users to have to always populate this property. It also makes more sense IMO to do it this way regardless, as alias is optional, but render-keys are not. * Make commit.DocKey option type Makes it far clearer that it is optional * Use option for parser.limit Also flattens them, as the previous structure suggested if one was present then the other was mandatory. Also switches from int64 to uint64 as they should not be negative (and makes it more consitent with a few other similar props who use uint64). * Use option for parser.order * Remove '.'s from parser.order.fields No reason for this to exist, and will confuse anyone using it. * Use option for parser.GroupBy * Use option for parser.Filter * Cleanup commit-query branching Now using proper consts, and defined in the same location as everything else. * Remove unused param * Remove unused commit.GetRoot func * Remove unused mutation.GetRoot func * Tidy up parser.Root references Previous if was conceptually incorrect. None option added no value (select.Root now defaults to Object, which could be handy for users anyway). Also types the enum. * Move parser.types to client dir Also breaks up types.go in preparation for the moving of the rest of the model. * Rename parser.Query to parser.Request As agreed as a team, commit does not seek to rename all variables referencing this - that can be done later to save drowning this large PR with minor changes * Move request model out of parser package The parser package is now responisble for converting an gql-ast into a shared model. Soon it will be responisble for converting a string into the shared model. Mostly a copy-paste commit. --- client/request/aggregate.go | 29 ++ client/request/commit.go | 48 ++ .../types.go => client/request/consts.go | 78 +--- client/request/doc.go | 14 + client/request/field.go | 19 + client/request/filter.go | 20 + client/request/group.go | 15 + client/request/mutation.go | 57 +++ client/request/order.go | 31 ++ client/request/request.go | 23 + client/request/select.go | 100 +++++ core/doc.go | 4 +- core/enumerable/skip.go | 6 +- core/enumerable/take.go | 6 +- db/collection_delete.go | 4 +- db/collection_update.go | 10 +- mapper/commitSelect.go | 2 +- mapper/mapper.go | 391 +++++++--------- mapper/targetable.go | 12 +- planner/average.go | 6 +- planner/commit.go | 10 +- planner/delete.go | 2 +- planner/executor.go | 5 +- planner/group.go | 10 +- planner/limit.go | 8 +- planner/planner.go | 22 +- planner/select.go | 29 +- planner/sum.go | 8 +- planner/top.go | 8 +- planner/update.go | 2 +- query/graphql/parser/commit.go | 100 ++--- query/graphql/parser/filter.go | 43 +- query/graphql/parser/mutation.go | 106 ++--- query/graphql/parser/query.go | 422 ++++++++---------- query/graphql/schema/descriptions.go | 10 +- query/graphql/schema/generate.go | 88 ++-- query/graphql/schema/types/commits.go | 10 +- .../query/explain/group_with_limit_test.go | 24 +- .../query/explain/with_limit_test.go | 40 +- 39 files changed, 948 insertions(+), 874 deletions(-) create mode 100644 client/request/aggregate.go create mode 100644 client/request/commit.go rename query/graphql/parser/types/types.go => client/request/consts.go (62%) create mode 100644 client/request/doc.go create mode 100644 client/request/field.go create mode 100644 client/request/filter.go create mode 100644 client/request/group.go create mode 100644 client/request/mutation.go create mode 100644 client/request/order.go create mode 100644 client/request/request.go create mode 100644 client/request/select.go diff --git a/client/request/aggregate.go b/client/request/aggregate.go new file mode 100644 index 0000000000..3f6ade72a7 --- /dev/null +++ b/client/request/aggregate.go @@ -0,0 +1,29 @@ +// Copyright 2022 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 request + +import "github.com/sourcenetwork/defradb/client" + +type Aggregate struct { + Field + + Targets []*AggregateTarget +} + +type AggregateTarget struct { + HostName string + ChildName client.Option[string] + + Limit client.Option[uint64] + Offset client.Option[uint64] + OrderBy client.Option[OrderBy] + Filter client.Option[Filter] +} diff --git a/client/request/commit.go b/client/request/commit.go new file mode 100644 index 0000000000..725ed16ddd --- /dev/null +++ b/client/request/commit.go @@ -0,0 +1,48 @@ +// Copyright 2022 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 request + +import "github.com/sourcenetwork/defradb/client" + +var ( + _ Selection = (*CommitSelect)(nil) +) + +type CommitSelect struct { + Field + + DocKey client.Option[string] + FieldName client.Option[string] + Cid client.Option[string] + Depth client.Option[uint64] + + Limit client.Option[uint64] + Offset client.Option[uint64] + OrderBy client.Option[OrderBy] + GroupBy client.Option[GroupBy] + + Fields []Selection +} + +func (c CommitSelect) ToSelect() *Select { + return &Select{ + Field: Field{ + Name: c.Name, + Alias: c.Alias, + }, + Limit: c.Limit, + Offset: c.Offset, + OrderBy: c.OrderBy, + GroupBy: c.GroupBy, + Fields: c.Fields, + Root: CommitSelection, + } +} diff --git a/query/graphql/parser/types/types.go b/client/request/consts.go similarity index 62% rename from query/graphql/parser/types/types.go rename to client/request/consts.go index 8254e29e8f..0d61d53cdd 100644 --- a/query/graphql/parser/types/types.go +++ b/client/request/consts.go @@ -8,65 +8,20 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -/* -Package types defines the GraphQL types used by the query service. -*/ -package types - -import "github.com/graphql-go/graphql/language/ast" - -type ( - OrderDirection string - - SelectionType int - - // Enum for different types of read Select queries - SelectQueryType int - - OrderCondition struct { - // field may be a compound field statement - // since the order statement allows ordering on - // sub objects. - // - // Given the statement: {order: {author: {birthday: DESC}}} - // The field value would be "author.birthday" - // and the direction would be "DESC" - Field string - Direction OrderDirection - } - - GroupBy struct { - Fields []string - } - - OrderBy struct { - Conditions []OrderCondition - Statement *ast.ObjectValue - } - - Limit struct { - Limit int64 - Offset int64 - } - - OptionalDocKeys struct { - HasValue bool - Value []string - } -) +package request const ( // GQL special field, returns the host object's type name // https://spec.graphql.org/October2021/#sec-Type-Name-Introspection TypeNameFieldName = "__typename" - Cid = "cid" - Data = "data" - DocKey = "dockey" - DocKeys = "dockeys" - Field = "field" - Id = "id" - Ids = "ids" + Cid = "cid" + Data = "data" + DocKey = "dockey" + DocKeys = "dockeys" + FieldName = "field" + Id = "id" + Ids = "ids" FilterClause = "filter" GroupByClause = "groupBy" @@ -85,6 +40,7 @@ const ( ExplainLabel = "explain" LatestCommitsQueryName = "latestCommits" + CommitsQueryName = "commits" CommitTypeName = "Commit" LinksFieldName = "links" @@ -99,17 +55,6 @@ const ( DESC = OrderDirection("DESC") ) -const ( - ScanQuery = iota - VersionedScanQuery -) - -const ( - NoneSelection = iota - ObjectSelection - CommitSelection -) - var ( NameToOrderDirection = map[string]OrderDirection{ string(ASC): ASC, @@ -132,6 +77,11 @@ var ( AverageFieldName: {}, } + CommitQueries = map[string]struct{}{ + LatestCommitsQueryName: {}, + CommitsQueryName: {}, + } + VersionFields = []string{ HeightFieldName, CidFieldName, diff --git a/client/request/doc.go b/client/request/doc.go new file mode 100644 index 0000000000..7cbd2a31bc --- /dev/null +++ b/client/request/doc.go @@ -0,0 +1,14 @@ +// Copyright 2022 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 request defines the GraphQL types used by the query service. +*/ +package request diff --git a/client/request/field.go b/client/request/field.go new file mode 100644 index 0000000000..e139a1bc8b --- /dev/null +++ b/client/request/field.go @@ -0,0 +1,19 @@ +// Copyright 2022 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 request + +import "github.com/sourcenetwork/defradb/client" + +// Field implements Selection +type Field struct { + Name string + Alias client.Option[string] +} diff --git a/client/request/filter.go b/client/request/filter.go new file mode 100644 index 0000000000..bf32713844 --- /dev/null +++ b/client/request/filter.go @@ -0,0 +1,20 @@ +// Copyright 2022 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 request + +// Filter contains the parsed condition map to be +// run by the Filter Evaluator. +// @todo: Cache filter structure for faster condition +// evaluation. +type Filter struct { + // parsed filter conditions + Conditions map[string]any +} diff --git a/client/request/group.go b/client/request/group.go new file mode 100644 index 0000000000..e2fd977a00 --- /dev/null +++ b/client/request/group.go @@ -0,0 +1,15 @@ +// Copyright 2022 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 request + +type GroupBy struct { + Fields []string +} diff --git a/client/request/mutation.go b/client/request/mutation.go new file mode 100644 index 0000000000..f3c1441b5c --- /dev/null +++ b/client/request/mutation.go @@ -0,0 +1,57 @@ +// Copyright 2022 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 request + +import "github.com/sourcenetwork/defradb/client" + +type MutationType int + +const ( + NoneMutationType = MutationType(iota) + CreateObjects + UpdateObjects + DeleteObjects +) + +// Mutation is a field on the MutationType +// of a graphql query. It includes all the possible +// arguments and all +// +// @todo: Change name to ObjectMutation to indicate +// generated object mutation actions +type Mutation struct { + Field + Type MutationType + + // Schema is the target schema/collection + // if this mutation is on an object. + Schema string + + IDs client.Option[[]string] + Filter client.Option[Filter] + Data string + + Fields []Selection +} + +// ToSelect returns a basic Select object, with the same Name, Alias, and Fields as +// the Mutation object. Used to create a Select planNode for the mutation return objects. +func (m Mutation) ToSelect() *Select { + return &Select{ + Field: Field{ + Name: m.Schema, + Alias: m.Alias, + }, + Fields: m.Fields, + DocKeys: m.IDs, + Filter: m.Filter, + } +} diff --git a/client/request/order.go b/client/request/order.go new file mode 100644 index 0000000000..1fff3953f1 --- /dev/null +++ b/client/request/order.go @@ -0,0 +1,31 @@ +// Copyright 2022 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 request + +type ( + OrderDirection string + + OrderCondition struct { + // field may be a compound field statement + // since the order statement allows ordering on + // sub objects. + // + // Given the statement: {order: {author: {birthday: DESC}}} + // The field value would be "author.birthday" + // and the direction would be "DESC" + Fields []string + Direction OrderDirection + } + + OrderBy struct { + Conditions []OrderCondition + } +) diff --git a/client/request/request.go b/client/request/request.go new file mode 100644 index 0000000000..372d2cc40a --- /dev/null +++ b/client/request/request.go @@ -0,0 +1,23 @@ +// Copyright 2022 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 request + +type Request struct { + Queries []*OperationDefinition + Mutations []*OperationDefinition +} + +type Selection any + +type OperationDefinition struct { + Selections []Selection + IsExplain bool +} diff --git a/client/request/select.go b/client/request/select.go new file mode 100644 index 0000000000..719b3bc686 --- /dev/null +++ b/client/request/select.go @@ -0,0 +1,100 @@ +// Copyright 2022 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 request + +import "github.com/sourcenetwork/defradb/client" + +type SelectionType int + +const ( + ObjectSelection SelectionType = iota + CommitSelection +) + +// Select is a complex Field with strong typing +// It used for sub types in a query. Includes +// fields, and query arguments like filters, +// limits, etc. +type Select struct { + Field + + DocKeys client.Option[[]string] + CID client.Option[string] + + // Root is the top level query parsed type + Root SelectionType + + Limit client.Option[uint64] + Offset client.Option[uint64] + OrderBy client.Option[OrderBy] + GroupBy client.Option[GroupBy] + Filter client.Option[Filter] + + Fields []Selection +} + +func (s *Select) Validate() []error { + result := []error{} + + result = append(result, s.validateShallow()...) + + for _, childSelection := range s.Fields { + switch typedChildSelection := childSelection.(type) { + case *Select: + result = append(result, typedChildSelection.validateShallow()...) + default: + // Do nothing + } + } + + return result +} + +func (s *Select) validateShallow() []error { + result := []error{} + + result = append(result, s.validateGroupBy()...) + + return result +} + +func (s *Select) validateGroupBy() []error { + result := []error{} + + if !s.GroupBy.HasValue() { + return result + } + + for _, childSelection := range s.Fields { + switch typedChildSelection := childSelection.(type) { + case *Field: + if typedChildSelection.Name == TypeNameFieldName { + // _typeName is permitted + continue + } + + var fieldExistsInGroupBy bool + for _, groupByField := range s.GroupBy.Value().Fields { + if typedChildSelection.Name == groupByField { + fieldExistsInGroupBy = true + break + } + } + if !fieldExistsInGroupBy { + result = append(result, client.NewErrSelectOfNonGroupField(typedChildSelection.Name)) + } + default: + // Do nothing + } + } + + return result +} diff --git a/core/doc.go b/core/doc.go index 8d2e4dfac8..dafc793ccd 100644 --- a/core/doc.go +++ b/core/doc.go @@ -15,7 +15,7 @@ package core import ( "github.com/sourcenetwork/defradb/client" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" + "github.com/sourcenetwork/defradb/client/request" ) const DocKeyFieldIndex int = 0 @@ -229,7 +229,7 @@ func (mapping *DocumentMapping) Add(index int, name string) { func (mapping *DocumentMapping) SetTypeName(typeName string) { index := mapping.GetNextIndex() - mapping.Add(index, parserTypes.TypeNameFieldName) + mapping.Add(index, request.TypeNameFieldName) mapping.typeInfo = client.Some(mappingTypeInfo{ Index: index, Name: typeName, diff --git a/core/enumerable/skip.go b/core/enumerable/skip.go index aa7d4b046e..2ad62d3af0 100644 --- a/core/enumerable/skip.go +++ b/core/enumerable/skip.go @@ -12,14 +12,14 @@ package enumerable type enumerableSkip[T any] struct { source Enumerable[T] - offset int64 - count int64 + offset uint64 + count uint64 } // Skip creates an `Enumerable` from the given `Enumerable` and offset. The returned // `Enumerable` will skip through items until the number of items yielded from source // excedes the give offset. -func Skip[T any](source Enumerable[T], offset int64) Enumerable[T] { +func Skip[T any](source Enumerable[T], offset uint64) Enumerable[T] { return &enumerableSkip[T]{ source: source, offset: offset, diff --git a/core/enumerable/take.go b/core/enumerable/take.go index ac07283890..d9fdb51380 100644 --- a/core/enumerable/take.go +++ b/core/enumerable/take.go @@ -12,13 +12,13 @@ package enumerable type enumerableTake[T any] struct { source Enumerable[T] - limit int64 - count int64 + limit uint64 + count uint64 } // Take creates an `Enumerable` from the given `Enumerable` and limit. The returned // `Enumerable` will restrict the maximum number of items yielded to the given limit. -func Take[T any](source Enumerable[T], limit int64) Enumerable[T] { +func Take[T any](source Enumerable[T], limit uint64) Enumerable[T] { return &enumerableTake[T]{ source: source, limit: limit, diff --git a/db/collection_delete.go b/db/collection_delete.go index 844b14cc6e..8f88740ac7 100644 --- a/db/collection_delete.go +++ b/db/collection_delete.go @@ -22,11 +22,11 @@ import ( dag "github.com/ipfs/go-merkledag" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/merkle/clock" - "github.com/sourcenetwork/defradb/query/graphql/parser" ) var ( @@ -43,7 +43,7 @@ func (c *collection) DeleteWith( target any, ) (*client.DeleteResult, error) { switch t := target.(type) { - case string, map[string]any, *parser.Filter: + case string, map[string]any, *request.Filter: return c.DeleteWithFilter(ctx, t) case client.DocKey: return c.DeleteWithKey(ctx, t) diff --git a/db/collection_update.go b/db/collection_update.go index 41237e5e1f..099521e984 100644 --- a/db/collection_update.go +++ b/db/collection_update.go @@ -19,13 +19,13 @@ import ( "github.com/valyala/fastjson" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" "github.com/sourcenetwork/defradb/planner" "github.com/sourcenetwork/defradb/query/graphql/parser" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) var ( @@ -47,7 +47,7 @@ func (c *collection) UpdateWith( updater string, ) (*client.UpdateResult, error) { switch t := target.(type) { - case string, map[string]any, *parser.Filter: + case string, map[string]any, *request.Filter: return c.UpdateWithFilter(ctx, t, updater) case client.DocKey: return c.UpdateWithKey(ctx, t, updater) @@ -276,7 +276,7 @@ func (c *collection) updateWithFilter( } // add successful updated doc to results - results.DocKeys = append(results.DocKeys, doc[parserTypes.DocKeyFieldName].(string)) + results.DocKeys = append(results.DocKeys, doc[request.DocKeyFieldName].(string)) results.Count++ } @@ -606,7 +606,7 @@ func (c *collection) makeSelectionQuery( if fval == "" { return nil, errors.New("invalid filter") } - var p *parser.Filter + var p client.Option[request.Filter] p, err = parser.NewFilterFromString(fval) if err != nil { return nil, err @@ -656,7 +656,7 @@ func (c *collection) makeSelectLocal(filter *mapper.Filter, mapping *core.Docume func (c *collection) createMapping() *core.DocumentMapping { mapping := core.NewDocumentMapping() - mapping.Add(core.DocKeyFieldIndex, parserTypes.DocKeyFieldName) + mapping.Add(core.DocKeyFieldIndex, request.DocKeyFieldName) for _, fd := range c.Schema().Fields { if fd.IsObject() { continue diff --git a/mapper/commitSelect.go b/mapper/commitSelect.go index 2b5967b965..0d15e7958d 100644 --- a/mapper/commitSelect.go +++ b/mapper/commitSelect.go @@ -20,7 +20,7 @@ type CommitSelect struct { Select // The key of the target document for which to get commits for. - DocKey string + DocKey client.Option[string] // The field for which commits have been requested. FieldName client.Option[string] diff --git a/mapper/mapper.go b/mapper/mapper.go index db11f51844..ceada747ba 100644 --- a/mapper/mapper.go +++ b/mapper/mapper.go @@ -14,26 +14,22 @@ import ( "context" "fmt" "reflect" - "strconv" "strings" - "github.com/graphql-go/graphql/language/ast" - "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/core/enumerable" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" - "github.com/sourcenetwork/defradb/query/graphql/parser" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) // ToSelect converts the given [parser.Select] into a [Select]. // // In the process of doing so it will construct the document map required to access the data // yielded by the [Select]. -func ToSelect(ctx context.Context, txn datastore.Txn, parsed *parser.Select) (*Select, error) { +func ToSelect(ctx context.Context, txn datastore.Txn, parsed *request.Select) (*Select, error) { descriptionsRepo := NewDescriptionsRepo(ctx, txn) // the top-level select will always have index=0, and no parent collection name return toSelect(descriptionsRepo, 0, parsed, "") @@ -46,7 +42,7 @@ func ToSelect(ctx context.Context, txn datastore.Txn, parsed *parser.Select) (*S func toSelect( descriptionsRepo *DescriptionsRepo, thisIndex int, - parsed *parser.Select, + parsed *request.Select, parentCollectionName string, ) (*Select, error) { collectionName, err := getCollectionName(descriptionsRepo, parsed, parentCollectionName) @@ -90,10 +86,10 @@ func toSelect( } // If there is a groupby, and no inner group has been requested, we need to map the property here - if parsed.GroupBy != nil { - if _, isGroupFieldMapped := mapping.IndexesByName[parserTypes.GroupFieldName]; !isGroupFieldMapped { + if parsed.GroupBy.HasValue() { + if _, isGroupFieldMapped := mapping.IndexesByName[request.GroupFieldName]; !isGroupFieldMapped { index := mapping.GetNextIndex() - mapping.Add(index, parserTypes.GroupFieldName) + mapping.Add(index, request.GroupFieldName) } } @@ -111,23 +107,22 @@ func toSelect( func resolveOrderDependencies( descriptionsRepo *DescriptionsRepo, descName string, - source *parserTypes.OrderBy, + source client.Option[request.OrderBy], mapping *core.DocumentMapping, existingFields *[]Requestable, ) error { - if source == nil { + if !source.HasValue() { return nil } // If there is orderby, and any one of the condition fields that are join fields and have not been // requested, we need to map them here. - for _, condition := range source.Conditions { - fieldNames := strings.Split(condition.Field, ".") - if len(fieldNames) <= 1 { + for _, condition := range source.Value().Conditions { + if len(condition.Fields) <= 1 { continue } - joinField := fieldNames[0] + joinField := condition.Fields[0] // Check if the join field is already mapped, if not then map it. if isOrderJoinFieldMapped := len(mapping.IndexesByName[joinField]) != 0; !isOrderJoinFieldMapped { @@ -135,7 +130,11 @@ func resolveOrderDependencies( mapping.Add(index, joinField) // Resolve the inner child fields and get it's mapping. - dummyJoinFieldSelect := parser.Select{Name: joinField} + dummyJoinFieldSelect := request.Select{ + Field: request.Field{ + Name: joinField, + }, + } innerSelect, err := toSelect(descriptionsRepo, index, &dummyJoinFieldSelect, descName) if err != nil { return err @@ -156,7 +155,7 @@ func resolveOrderDependencies( // append the new target field as well as the aggregate. The mapping will also be // updated with any new fields/aggregates. func resolveAggregates( - parsed *parser.Select, + parsed *request.Select, aggregates []*aggregateRequest, inputFields []Requestable, mapping *core.DocumentMapping, @@ -184,12 +183,12 @@ func resolveAggregates( fieldDesc, isField := desc.GetField(target.hostExternalName) if isField && !fieldDesc.IsObject() { var order *OrderBy - if target.order != nil && len(target.order.Conditions) > 0 { + if target.order.HasValue() && len(target.order.Value().Conditions) > 0 { // For inline arrays the order element will consist of just a direction order = &OrderBy{ Conditions: []OrderCondition{ { - Direction: SortDirection(target.order.Conditions[0].Direction), + Direction: SortDirection(target.order.Value().Conditions[0].Direction), }, }, } @@ -226,9 +225,11 @@ func resolveAggregates( // If a matching host is not found, we need to construct and add it. index := mapping.GetNextIndex() - dummyParsed := &parser.Select{ + dummyParsed := &request.Select{ Root: parsed.Root, - Name: target.hostExternalName, + Field: request.Field{ + Name: target.hostExternalName, + }, } childCollectionName, err := getCollectionName(descriptionsRepo, dummyParsed, desc.Name) @@ -348,9 +349,9 @@ func fieldAt(fields []Requestable, index int) Requestable { // aggregateDependencies maps aggregate names to the names of any aggregates // that they may be dependent on. var aggregateDependencies = map[string][]string{ - parserTypes.AverageFieldName: { - parserTypes.CountFieldName, - parserTypes.SumFieldName, + request.AverageFieldName: { + request.CountFieldName, + request.SumFieldName, }, } @@ -379,7 +380,7 @@ func appendUnderlyingAggregates( for _, target := range aggregate.targets { if target.childExternalName != "" { - if _, isAggregate := parserTypes.Aggregates[target.childExternalName]; isAggregate { + if _, isAggregate := request.Aggregates[target.childExternalName]; isAggregate { continue } } @@ -438,32 +439,14 @@ func appendIfNotExists( // and aggregateRequests from the given parsed.Fields slice. It also mutates the // consumed mapping data. func getRequestables( - parsed *parser.Select, + parsed *request.Select, mapping *core.DocumentMapping, desc *client.CollectionDescription, descriptionsRepo *DescriptionsRepo, ) (fields []Requestable, aggregates []*aggregateRequest, err error) { - // If this parser.Select is itself an aggregate, we need to append the - // relevent info here as if it was a field of its own (due to a quirk of - // the parser package). - if _, isAggregate := parserTypes.Aggregates[parsed.Name]; isAggregate { - index := mapping.GetNextIndex() - aggregateReq, err := getAggregateRequests(index, parsed) - if err != nil { - return nil, nil, err - } - - mapping.RenderKeys = append(mapping.RenderKeys, core.RenderKey{ - Index: index, - Key: parsed.Alias, - }) - mapping.Add(index, parsed.Name) - aggregates = append(aggregates, &aggregateReq) - } - for _, field := range parsed.Fields { switch f := field.(type) { - case *parser.Field: + case *request.Field: // We can map all fields to the first (and only index) // as they support no value modifiers (such as filters/limits/etc). // All fields should have already been mapped by getTopLevelInfo @@ -476,34 +459,36 @@ func getRequestables( mapping.RenderKeys = append(mapping.RenderKeys, core.RenderKey{ Index: index, - Key: f.Alias, + Key: getRenderKey(f), }) - case *parser.Select: + case *request.Select: index := mapping.GetNextIndex() - // Aggregate targets are not known at this point, and must be evaluated - // after all requested fields have been evaluated - so we note which - // aggregates have been requested and their targets here, before finalizing - // their evaluation later. - if _, isAggregate := parserTypes.Aggregates[f.Name]; isAggregate { - aggregateRequest, err := getAggregateRequests(index, f) - if err != nil { - return nil, nil, err - } + innerSelect, err := toSelect(descriptionsRepo, index, f, desc.Name) + if err != nil { + return nil, nil, err + } + fields = append(fields, innerSelect) + mapping.SetChildAt(index, &innerSelect.DocumentMapping) - aggregates = append(aggregates, &aggregateRequest) - } else { - innerSelect, err := toSelect(descriptionsRepo, index, f, desc.Name) - if err != nil { - return nil, nil, err - } - fields = append(fields, innerSelect) - mapping.SetChildAt(index, &innerSelect.DocumentMapping) + mapping.RenderKeys = append(mapping.RenderKeys, core.RenderKey{ + Index: index, + Key: getRenderKey(&f.Field), + }) + + mapping.Add(index, f.Name) + case *request.Aggregate: + index := mapping.GetNextIndex() + aggregateRequest, err := getAggregateRequests(index, f) + if err != nil { + return nil, nil, err } + aggregates = append(aggregates, &aggregateRequest) + mapping.RenderKeys = append(mapping.RenderKeys, core.RenderKey{ Index: index, - Key: f.Alias, + Key: getRenderKey(&f.Field), }) mapping.Add(index, f.Name) @@ -517,7 +502,14 @@ func getRequestables( return } -func getAggregateRequests(index int, aggregate *parser.Select) (aggregateRequest, error) { +func getRenderKey(field *request.Field) string { + if field.Alias.HasValue() { + return field.Alias.Value() + } + return field.Name +} + +func getAggregateRequests(index int, aggregate *request.Aggregate) (aggregateRequest, error) { aggregateTargets, err := getAggregateSources(aggregate) if err != nil { return aggregateRequest{}, err @@ -542,17 +534,17 @@ func getAggregateRequests(index int, aggregate *parser.Select) (aggregateRequest // if this is a commit request. func getCollectionName( descriptionsRepo *DescriptionsRepo, - parsed *parser.Select, + parsed *request.Select, parentCollectionName string, ) (string, error) { - if _, isAggregate := parserTypes.Aggregates[parsed.Name]; isAggregate { + if _, isAggregate := request.Aggregates[parsed.Name]; isAggregate { // This string is not used or referenced, its value is only there to aid debugging return "_topLevel", nil } - if parsed.Name == parserTypes.GroupFieldName { + if parsed.Name == request.GroupFieldName { return parentCollectionName, nil - } else if parsed.Root == parserTypes.CommitSelection { + } else if parsed.Root == request.CommitSelection { return parentCollectionName, nil } @@ -576,19 +568,19 @@ func getCollectionName( // getTopLevelInfo returns the collection description and maps the fields directly on the object. func getTopLevelInfo( descriptionsRepo *DescriptionsRepo, - parsed *parser.Select, + parsed *request.Select, collectionName string, ) (*core.DocumentMapping, *client.CollectionDescription, error) { mapping := core.NewDocumentMapping() - if _, isAggregate := parserTypes.Aggregates[parsed.Name]; isAggregate { + if _, isAggregate := request.Aggregates[parsed.Name]; isAggregate { // If this is a (top-level) aggregate, then it will have no collection // description, and no top-level fields, so we return an empty mapping only return mapping, &client.CollectionDescription{}, nil } - if parsed.Root != parserTypes.CommitSelection { - mapping.Add(core.DocKeyFieldIndex, parserTypes.DocKeyFieldName) + if parsed.Root == request.ObjectSelection { + mapping.Add(core.DocKeyFieldIndex, request.DocKeyFieldName) desc, err := descriptionsRepo.getCollectionDesc(collectionName) if err != nil { @@ -612,22 +604,22 @@ func getTopLevelInfo( return mapping, &desc, nil } - if parsed.Name == parserTypes.LinksFieldName { - for i, f := range parserTypes.LinksFields { + if parsed.Name == request.LinksFieldName { + for i, f := range request.LinksFields { mapping.Add(i, f) } // Setting the type name must be done after adding the fields, as // the typeName index is dynamic, but the field indexes are not - mapping.SetTypeName(parserTypes.LinksFieldName) + mapping.SetTypeName(request.LinksFieldName) } else { - for i, f := range parserTypes.VersionFields { + for i, f := range request.VersionFields { mapping.Add(i, f) } // Setting the type name must be done after adding the fields, as // the typeName index is dynamic, but the field indexes are not - mapping.SetTypeName(parserTypes.CommitTypeName) + mapping.SetTypeName(request.CommitTypeName) } return mapping, &client.CollectionDescription{}, nil @@ -636,18 +628,18 @@ func getTopLevelInfo( func resolveFilterDependencies( descriptionsRepo *DescriptionsRepo, parentCollectionName string, - source *parser.Filter, + source client.Option[request.Filter], mapping *core.DocumentMapping, existingFields []Requestable, ) ([]Requestable, error) { - if source == nil { + if !source.HasValue() { return nil, nil } return resolveInnerFilterDependencies( descriptionsRepo, parentCollectionName, - source.Conditions, + source.Value().Conditions, mapping, existingFields, ) @@ -663,7 +655,7 @@ func resolveInnerFilterDependencies( newFields := []Requestable{} for key := range source { - if strings.HasPrefix(key, "_") && key != parserTypes.DocKeyFieldName { + if strings.HasPrefix(key, "_") && key != request.DocKeyFieldName { continue } @@ -672,8 +664,10 @@ func resolveInnerFilterDependencies( if !propertyMapped { index := mapping.GetNextIndex() - dummyParsed := &parser.Select{ - Name: key, + dummyParsed := &request.Select{ + Field: request.Field{ + Name: key, + }, } childCollectionName, err := getCollectionName(descriptionsRepo, dummyParsed, parentCollectionName) @@ -726,8 +720,10 @@ func resolveInnerFilterDependencies( continue } - dummyParsed := &parser.Select{ - Name: key, + dummyParsed := &request.Select{ + Field: request.Field{ + Name: key, + }, } childCollectionName, err := getCollectionName(descriptionsRepo, dummyParsed, parentCollectionName) @@ -779,11 +775,11 @@ func resolveInnerFilterDependencies( return newFields, nil } -// ToCommitSelect converts the given [parser.CommitSelect] into a [CommitSelect]. +// ToCommitSelect converts the given [request.CommitSelect] into a [CommitSelect]. // // In the process of doing so it will construct the document map required to access the data // yielded by the [Select] embedded in the [CommitSelect]. -func ToCommitSelect(ctx context.Context, txn datastore.Txn, parsed *parser.CommitSelect) (*CommitSelect, error) { +func ToCommitSelect(ctx context.Context, txn datastore.Txn, parsed *request.CommitSelect) (*CommitSelect, error) { underlyingSelect, err := ToSelect(ctx, txn, parsed.ToSelect()) if err != nil { return nil, err @@ -797,11 +793,11 @@ func ToCommitSelect(ctx context.Context, txn datastore.Txn, parsed *parser.Commi }, nil } -// ToMutation converts the given [parser.Mutation] into a [Mutation]. +// ToMutation converts the given [request.Mutation] into a [Mutation]. // // In the process of doing so it will construct the document map required to access the data // yielded by the [Select] embedded in the [Mutation]. -func ToMutation(ctx context.Context, txn datastore.Txn, parsed *parser.Mutation) (*Mutation, error) { +func ToMutation(ctx context.Context, txn datastore.Txn, parsed *request.Mutation) (*Mutation, error) { underlyingSelect, err := ToSelect(ctx, txn, parsed.ToSelect()) if err != nil { return nil, err @@ -814,41 +810,41 @@ func ToMutation(ctx context.Context, txn datastore.Txn, parsed *parser.Mutation) }, nil } -func toTargetable(index int, parsed *parser.Select, docMap *core.DocumentMapping) Targetable { +func toTargetable(index int, parsed *request.Select, docMap *core.DocumentMapping) Targetable { return Targetable{ Field: toField(index, parsed), - DocKeys: OptionalDocKeys(parsed.DocKeys), + DocKeys: parsed.DocKeys, Filter: ToFilter(parsed.Filter, docMap), - Limit: toLimit(parsed.Limit), + Limit: toLimit(parsed.Limit, parsed.Offset), GroupBy: toGroupBy(parsed.GroupBy, docMap), OrderBy: toOrderBy(parsed.OrderBy, docMap), } } -func toField(index int, parsed *parser.Select) Field { +func toField(index int, parsed *request.Select) Field { return Field{ Index: index, Name: parsed.Name, } } -// ToFilter converts the given `source` parser filter to a Filter using the given mapping. +// ToFilter converts the given `source` request filter to a Filter using the given mapping. // // Any requestables identified by name will be converted to being identified by index instead. -func ToFilter(source *parser.Filter, mapping *core.DocumentMapping) *Filter { - if source == nil { +func ToFilter(source client.Option[request.Filter], mapping *core.DocumentMapping) *Filter { + if !source.HasValue() { return nil } - conditions := make(map[connor.FilterKey]any, len(source.Conditions)) + conditions := make(map[connor.FilterKey]any, len(source.Value().Conditions)) - for sourceKey, sourceClause := range source.Conditions { + for sourceKey, sourceClause := range source.Value().Conditions { key, clause := toFilterMap(sourceKey, sourceClause, mapping) conditions[key] = clause } return &Filter{ Conditions: conditions, - ExternalConditions: source.Conditions, + ExternalConditions: source.Value().Conditions, } } @@ -861,7 +857,7 @@ func toFilterMap( sourceClause any, mapping *core.DocumentMapping, ) (connor.FilterKey, any) { - if strings.HasPrefix(sourceKey, "_") && sourceKey != parserTypes.DocKeyFieldName { + if strings.HasPrefix(sourceKey, "_") && sourceKey != request.DocKeyFieldName { key := &Operator{ Operation: sourceKey, } @@ -921,24 +917,34 @@ func toFilterMap( } } -func toLimit(source *parserTypes.Limit) *Limit { - if source == nil { +func toLimit(limit client.Option[uint64], offset client.Option[uint64]) *Limit { + var limitValue uint64 + var offsetValue uint64 + if !limit.HasValue() && !offset.HasValue() { return nil } + if limit.HasValue() { + limitValue = limit.Value() + } + + if offset.HasValue() { + offsetValue = offset.Value() + } + return &Limit{ - Limit: source.Limit, - Offset: source.Offset, + Limit: limitValue, + Offset: offsetValue, } } -func toGroupBy(source *parserTypes.GroupBy, mapping *core.DocumentMapping) *GroupBy { - if source == nil { +func toGroupBy(source client.Option[request.GroupBy], mapping *core.DocumentMapping) *GroupBy { + if !source.HasValue() { return nil } - fields := make([]Field, len(source.Fields)) - for i, fieldName := range source.Fields { + fields := make([]Field, len(source.Value().Fields)) + for i, fieldName := range source.Value().Fields { // If there are multiple properties of the same name we can just take the first as // we have no other reasonable way of identifying which property they mean if multiple // consumer specified requestables are available. Aggregate dependencies should not @@ -956,24 +962,23 @@ func toGroupBy(source *parserTypes.GroupBy, mapping *core.DocumentMapping) *Grou } } -func toOrderBy(source *parserTypes.OrderBy, mapping *core.DocumentMapping) *OrderBy { - if source == nil { +func toOrderBy(source client.Option[request.OrderBy], mapping *core.DocumentMapping) *OrderBy { + if !source.HasValue() { return nil } - conditions := make([]OrderCondition, len(source.Conditions)) - for conditionIndex, condition := range source.Conditions { - fields := strings.Split(condition.Field, ".") - fieldIndexes := make([]int, len(fields)) + conditions := make([]OrderCondition, len(source.Value().Conditions)) + for conditionIndex, condition := range source.Value().Conditions { + fieldIndexes := make([]int, len(condition.Fields)) currentMapping := mapping - for fieldIndex, field := range fields { + for fieldIndex, field := range condition.Fields { // If there are multiple properties of the same name we can just take the first as // we have no other reasonable way of identifying which property they mean if multiple // consumer specified requestables are available. Aggregate dependencies should not // impact this as they are added after selects. firstFieldIndex := currentMapping.FirstIndexOfName(field) fieldIndexes[fieldIndex] = firstFieldIndex - if fieldIndex != len(fields)-1 { + if fieldIndex != len(condition.Fields)-1 { // no need to do this for the last (and will panic) currentMapping = currentMapping.ChildMappings[firstFieldIndex] } @@ -1152,131 +1157,33 @@ type aggregateRequestTarget struct { childExternalName string // The aggregate filter specified by the consumer for this target. Optional. - filter *parser.Filter + filter client.Option[request.Filter] // The aggregate limit-offset specified by the consumer for this target. Optional. limit *Limit // The order in which items should be aggregated. Affects results when used with // limit. Optional. - order *parserTypes.OrderBy + order client.Option[request.OrderBy] } // Returns the source of the aggregate as requested by the consumer -func getAggregateSources(field *parser.Select) ([]*aggregateRequestTarget, error) { - targets := make([]*aggregateRequestTarget, len(field.Statement.Arguments)) - - for i, argument := range field.Statement.Arguments { - switch argumentValue := argument.Value.GetValue().(type) { - case string: - targets[i] = &aggregateRequestTarget{ - hostExternalName: argumentValue, - } - case []*ast.ObjectField: - hostExternalName := argument.Name.Value - var childExternalName string - var filter *parser.Filter - var limit *Limit - var order *parserTypes.OrderBy - - fieldArg, hasFieldArg := tryGet(argumentValue, parserTypes.Field) - if hasFieldArg { - if innerPathStringValue, isString := fieldArg.Value.GetValue().(string); isString { - childExternalName = innerPathStringValue - } - } - - filterArg, hasFilterArg := tryGet(argumentValue, parserTypes.FilterClause) - if hasFilterArg { - var err error - filter, err = parser.NewFilter(filterArg.Value.(*ast.ObjectValue)) - if err != nil { - return nil, err - } - } - - limitArg, hasLimitArg := tryGet(argumentValue, parserTypes.LimitClause) - offsetArg, hasOffsetArg := tryGet(argumentValue, parserTypes.OffsetClause) - var limitValue int64 - var offsetValue int64 - if hasLimitArg { - var err error - limitValue, err = strconv.ParseInt(limitArg.Value.(*ast.IntValue).Value, 10, 64) - if err != nil { - return nil, err - } - } - - if hasOffsetArg { - var err error - offsetValue, err = strconv.ParseInt(offsetArg.Value.(*ast.IntValue).Value, 10, 64) - if err != nil { - return nil, err - } - } - - if hasLimitArg || hasOffsetArg { - limit = &Limit{ - Limit: limitValue, - Offset: offsetValue, - } - } - - orderArg, hasOrderArg := tryGet(argumentValue, parserTypes.OrderClause) - if hasOrderArg { - switch orderArgValue := orderArg.Value.(type) { - case *ast.EnumValue: - // For inline arrays the order arg will be a simple enum declaring the order direction - orderDirectionString := orderArgValue.Value - orderDirection := parserTypes.OrderDirection(orderDirectionString) - - order = &parserTypes.OrderBy{ - Conditions: []parserTypes.OrderCondition{ - { - Direction: orderDirection, - }, - }, - } - - case *ast.ObjectValue: - // For relations the order arg will be the complex order object as used by the host object - // for non-aggregate ordering - - // We use the parser package parsing for convienience here - orderConditions, err := parser.ParseConditionsInOrder(orderArgValue) - if err != nil { - return nil, err - } - - order = &parserTypes.OrderBy{ - Conditions: orderConditions, - Statement: orderArgValue, - } - } - } - - targets[i] = &aggregateRequestTarget{ - hostExternalName: hostExternalName, - childExternalName: childExternalName, - filter: filter, - limit: limit, - order: order, - } +func getAggregateSources(field *request.Aggregate) ([]*aggregateRequestTarget, error) { + targets := make([]*aggregateRequestTarget, len(field.Targets)) + + for i, target := range field.Targets { + targets[i] = &aggregateRequestTarget{ + hostExternalName: target.HostName, + childExternalName: target.ChildName.Value(), + filter: target.Filter, + limit: toLimit(target.Limit, target.Offset), + order: target.OrderBy, } } return targets, nil } -func tryGet(fields []*ast.ObjectField, name string) (*ast.ObjectField, bool) { - for _, field := range fields { - if field.Name.Value == name { - return field, true - } - } - return nil, false -} - // tryGetMatchingAggregate scans the given collection for aggregates with the given name and targets. // // Will return the matching target and true if one is found, otherwise will return false. @@ -1305,20 +1212,20 @@ collectionLoop: continue collectionLoop } - if target.filter == nil && potentialMatchingTarget.filter != nil { + if !target.filter.HasValue() && potentialMatchingTarget.filter.HasValue() { continue collectionLoop } - if potentialMatchingTarget.filter == nil && target.filter != nil { + if !potentialMatchingTarget.filter.HasValue() && target.filter.HasValue() { continue collectionLoop } - if target.filter == nil && potentialMatchingTarget.filter == nil { + if !target.filter.HasValue() && !potentialMatchingTarget.filter.HasValue() { // target matches, so continue the `target` loop and check the remaining. continue } - if !reflect.DeepEqual(target.filter.Conditions, potentialMatchingTarget.filter.Conditions) { + if !reflect.DeepEqual(target.filter.Value().Conditions, potentialMatchingTarget.filter.Value().Conditions) { continue collectionLoop } } @@ -1365,23 +1272,23 @@ func tryGetTarget( // appendNotNilFilter appends a not nil filter for the given child field // to the given Select. func appendNotNilFilter(field *aggregateRequestTarget, childField string) { - if field.filter == nil { - field.filter = &parser.Filter{} - } - - if field.filter.Conditions == nil { - field.filter.Conditions = map[string]any{} + if !field.filter.HasValue() || field.filter.Value().Conditions == nil { + field.filter = client.Some( + request.Filter{ + Conditions: map[string]any{}, + }, + ) } var childBlock any var hasChildBlock bool if childField == "" { - childBlock = field.filter.Conditions + childBlock = field.filter.Value().Conditions } else { - childBlock, hasChildBlock = field.filter.Conditions[childField] + childBlock, hasChildBlock = field.filter.Value().Conditions[childField] if !hasChildBlock { childBlock = map[string]any{} - field.filter.Conditions[childField] = childBlock + field.filter.Value().Conditions[childField] = childBlock } } diff --git a/mapper/targetable.go b/mapper/targetable.go index 0ec2edf16c..ee8e84e8ff 100644 --- a/mapper/targetable.go +++ b/mapper/targetable.go @@ -11,6 +11,7 @@ package mapper import ( + "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/connor" "github.com/sourcenetwork/defradb/core" ) @@ -88,11 +89,11 @@ func NewFilter() *Filter { // and which records will be returned from a query. type Limit struct { // The maximum number of records that can be returned from a query. - Limit int64 + Limit uint64 // The offset from which counting towards the Limit will begin. // Before records before the Offset will not be returned. - Offset int64 + Offset uint64 } // GroupBy represents a grouping instruction on a query. @@ -124,11 +125,6 @@ type OrderBy struct { Conditions []OrderCondition } -type OptionalDocKeys struct { - HasValue bool - Value []string -} - // Targetable represents a targetable property. type Targetable struct { // The basic field information of this property. @@ -136,7 +132,7 @@ type Targetable struct { // A optional collection of docKeys that can be specified to restrict results // to belonging to this set. - DocKeys OptionalDocKeys + DocKeys client.Option[[]string] // An optional filter, that can be specified to restrict results to documents // that satisfies all of its conditions. diff --git a/planner/average.go b/planner/average.go index 3b0aa8ed8f..edd9fa0588 100644 --- a/planner/average.go +++ b/planner/average.go @@ -13,10 +13,10 @@ package planner import ( "fmt" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) type averageNode struct { @@ -38,9 +38,9 @@ func (p *Planner) Average( for _, dependency := range field.Dependencies { switch dependency.Name { - case parserTypes.CountFieldName: + case request.CountFieldName: countField = dependency - case parserTypes.SumFieldName: + case request.SumFieldName: sumField = dependency default: return nil, errors.New(fmt.Sprintf("Unknown dependency, name: %s", dependency.Name)) diff --git a/planner/commit.go b/planner/commit.go index ce64aace55..1a360353b6 100644 --- a/planner/commit.go +++ b/planner/commit.go @@ -17,11 +17,11 @@ import ( ipld "github.com/ipfs/go-ipld-format" dag "github.com/ipfs/go-merkledag" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/db/fetcher" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) type dagScanNode struct { @@ -62,8 +62,8 @@ func (n *dagScanNode) Kind() string { func (n *dagScanNode) Init() error { if len(n.spans.Value) == 0 { key := core.DataStoreKey{} - if n.parsed.DocKey != "" { - key = key.WithDocKey(n.parsed.DocKey) + if n.parsed.DocKey.HasValue() { + key = key.WithDocKey(n.parsed.DocKey.Value()) if n.parsed.FieldName.HasValue() { field := n.parsed.FieldName.Value() @@ -165,7 +165,7 @@ func (n *dagScanNode) Next() (bool, error) { if len(n.queuedCids) > 0 { currentCid = n.queuedCids[0] n.queuedCids = n.queuedCids[1:(len(n.queuedCids))] - } else if n.parsed.Cid.HasValue() && n.parsed.DocKey == "" { + } else if n.parsed.Cid.HasValue() && !n.parsed.DocKey.HasValue() { if n.visitedNodes[n.parsed.Cid.Value()] { // If the requested cid has been visited, we are done and should return false return false, nil @@ -308,7 +308,7 @@ func (n *dagScanNode) dagBlockToNodeDoc(block blocks.Block) (core.Doc, []*ipld.L heads := make([]*ipld.Link, 0) // links - linksIndexes := n.parsed.DocumentMapping.IndexesByName[parserTypes.LinksFieldName] + linksIndexes := n.parsed.DocumentMapping.IndexesByName[request.LinksFieldName] for _, linksIndex := range linksIndexes { links := make([]core.Doc, len(nd.Links())) diff --git a/planner/delete.go b/planner/delete.go index c6d04bb94d..bcb43f436e 100644 --- a/planner/delete.go +++ b/planner/delete.go @@ -106,7 +106,7 @@ func (p *Planner) DeleteDocs(parsed *mapper.Mutation) (planNode, error) { return &deleteNode{ p: p, filter: parsed.Filter, - ids: parsed.DocKeys.Value, + ids: parsed.DocKeys.Value(), collection: col.WithTxn(p.txn), source: slctNode, docMapper: docMapper{&parsed.DocumentMapping}, diff --git a/planner/executor.go b/planner/executor.go index e2fc5c0ef0..7a3ae43345 100644 --- a/planner/executor.go +++ b/planner/executor.go @@ -19,6 +19,7 @@ import ( "github.com/graphql-go/graphql/language/source" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" @@ -82,13 +83,13 @@ func (e *QueryExecutor) MakePlanFromParser( ctx context.Context, db client.DB, txn datastore.Txn, - query *parser.Query, + query *request.Request, ) (planNode, error) { planner := makePlanner(ctx, db, txn) return planner.makePlan(query) } -func (e *QueryExecutor) ParseRequestString(request string) (*parser.Query, error) { +func (e *QueryExecutor) ParseRequestString(request string) (*request.Request, error) { source := source.NewSource(&source.Source{ Body: []byte(request), Name: "GraphQL request", diff --git a/planner/group.go b/planner/group.go index eead2b6723..5993b6584d 100644 --- a/planner/group.go +++ b/planner/group.go @@ -13,10 +13,10 @@ package planner import ( "fmt" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) // A node responsible for the grouping of documents by a given selection of fields. @@ -55,7 +55,7 @@ func (p *Planner) GroupBy(n *mapper.GroupBy, parsed *mapper.Select, childSelects dataSources = append( dataSources, // If there are no child selects, then we just take the first field index of name _group - newDataSource(parsed.DocumentMapping.FirstIndexOfName(parserTypes.GroupFieldName)), + newDataSource(parsed.DocumentMapping.FirstIndexOfName(request.GroupFieldName)), ) } @@ -147,10 +147,10 @@ func (n *groupNode) Next() (bool, error) { childDocs := subSelect.([]core.Doc) if childSelect.Limit != nil { - l := int64(len(childDocs)) + l := uint64(len(childDocs)) // We must hide all child documents before the offset - for i := int64(0); i < childSelect.Limit.Offset && i < l; i++ { + for i := uint64(0); i < childSelect.Limit.Offset && i < l; i++ { childDocs[i].Hidden = true } @@ -205,7 +205,7 @@ func (n *groupNode) Explain() (map[string]any, error) { // Get targetable attribute(s) of this child. - if c.DocKeys.HasValue { + if c.DocKeys.HasValue() { childExplainGraph["docKeys"] = c.DocKeys.Value } else { childExplainGraph["docKeys"] = nil diff --git a/planner/limit.go b/planner/limit.go index 5b07183cc1..f928e67247 100644 --- a/planner/limit.go +++ b/planner/limit.go @@ -23,9 +23,9 @@ type limitNode struct { p *Planner plan planNode - limit int64 - offset int64 - rowIndex int64 + limit uint64 + offset uint64 + rowIndex uint64 } // Limit creates a new limitNode initalized from the parser.Limit object. @@ -58,7 +58,7 @@ func (n *limitNode) Value() core.Doc { return n.plan.Value() } func (n *limitNode) Next() (bool, error) { // check if we're passed the limit - if n.limit != 0 && n.rowIndex-n.offset >= n.limit { + if n.limit != 0 && n.rowIndex >= n.limit+n.offset { return false, nil } diff --git a/planner/planner.go b/planner/planner.go index 07e3c69807..ccf802e640 100644 --- a/planner/planner.go +++ b/planner/planner.go @@ -15,13 +15,12 @@ import ( "fmt" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/datastore" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/logging" "github.com/sourcenetwork/defradb/mapper" - "github.com/sourcenetwork/defradb/query/graphql/parser" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) var ( @@ -104,7 +103,7 @@ func makePlanner(ctx context.Context, db client.DB, txn datastore.Txn) *Planner func (p *Planner) newPlan(stmt any) (planNode, error) { switch n := stmt.(type) { - case *parser.Query: + case *request.Request: if len(n.Queries) > 0 { return p.newPlan(n.Queries[0]) // @todo, handle multiple query statements } else if len(n.Mutations) > 0 { @@ -113,19 +112,19 @@ func (p *Planner) newPlan(stmt any) (planNode, error) { return nil, errors.New("query is missing query or mutation statements") } - case *parser.OperationDefinition: + case *request.OperationDefinition: if len(n.Selections) == 0 { return nil, errors.New("operationDefinition is missing selections") } return p.newPlan(n.Selections[0]) - case *parser.Select: + case *request.Select: m, err := mapper.ToSelect(p.ctx, p.txn, n) if err != nil { return nil, err } - if _, isAgg := parserTypes.Aggregates[n.Name]; isAgg { + if _, isAgg := request.Aggregates[n.Name]; isAgg { // If this Select is an aggregate, then it must be a top-level // aggregate and we need to resolve it within the context of a // top-level node. @@ -133,17 +132,18 @@ func (p *Planner) newPlan(stmt any) (planNode, error) { } return p.Select(m) + case *mapper.Select: return p.Select(n) - case *parser.CommitSelect: + case *request.CommitSelect: m, err := mapper.ToCommitSelect(p.ctx, p.txn, n) if err != nil { return nil, err } return p.CommitSelect(m) - case *parser.Mutation: + case *request.Mutation: m, err := mapper.ToMutation(p.ctx, p.txn, n) if err != nil { return nil, err @@ -459,7 +459,7 @@ func (p *Planner) explainRequest( topExplainGraph := []map[string]any{ { - parserTypes.ExplainLabel: explainGraph, + request.ExplainLabel: explainGraph, }, } @@ -507,7 +507,7 @@ func (p *Planner) executeRequest( // runRequest plans how to run the request, then attempts to run the request and returns the results. func (p *Planner) runRequest( ctx context.Context, - query *parser.Query, + query *request.Request, ) ([]map[string]any, error) { plan, err := p.makePlan(query) @@ -528,7 +528,7 @@ func (p *Planner) runRequest( } // MakePlan makes a plan from the parsed query. @TODO {defradb/issues/368}: Test this exported function. -func (p *Planner) MakePlan(query *parser.Query) (planNode, error) { +func (p *Planner) MakePlan(query *request.Request) (planNode, error) { return p.makePlan(query) } diff --git a/planner/select.go b/planner/select.go index b096d63191..37c81f5760 100644 --- a/planner/select.go +++ b/planner/select.go @@ -13,12 +13,13 @@ package planner import ( cid "github.com/ipfs/go-cid" + "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/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) /* @@ -115,7 +116,7 @@ type selectNode struct { // are defined in the subtype scan node. filter *mapper.Filter - docKeys mapper.OptionalDocKeys + docKeys client.Option[[]string] parsed *mapper.Select groupSelects []*mapper.Select @@ -153,9 +154,9 @@ func (n *selectNode) Next() (bool, error) { continue } - if n.docKeys.HasValue { + if n.docKeys.HasValue() { docKey := n.currentValue.GetKey() - for _, key := range n.docKeys.Value { + for _, key := range n.docKeys.Value() { if docKey == key { return true, nil } @@ -229,19 +230,19 @@ func (n *selectNode) initSource() ([]aggregateNode, error) { ) } spans := fetcher.NewVersionedSpan( - core.DataStoreKey{DocKey: n.parsed.DocKeys.Value[0]}, + core.DataStoreKey{DocKey: n.parsed.DocKeys.Value()[0]}, c, ) // @todo check len origScan.Spans(spans) - } else if n.parsed.DocKeys.HasValue { + } else if n.parsed.DocKeys.HasValue() { // If we *just* have a DocKey(s), run a FindByDocKey(s) optimization // if we have a FindByDockey filter, create a span for it // and propagate it to the scanNode // @todo: When running the optimizer, check if the filter object // contains a _key equality condition, and upgrade it to a point lookup // instead of a prefix scan + filter via the Primary Index (0), like here: - spans := make([]core.Span, len(n.parsed.DocKeys.Value)) - for i, docKey := range n.parsed.DocKeys.Value { + spans := make([]core.Span, len(n.parsed.DocKeys.Value())) + for i, docKey := range n.parsed.DocKeys.Value() { dockeyIndexKey := base.MakeDocKey(sourcePlan.info.collectionDescription, docKey) spans[i] = core.NewSpan(dockeyIndexKey, dockeyIndexKey.PrefixEnd()) } @@ -263,11 +264,11 @@ func (n *selectNode) initFields(parsed *mapper.Select) ([]aggregateNode, error) var aggregateError error switch f.Name { - case parserTypes.CountFieldName: + case request.CountFieldName: plan, aggregateError = n.p.Count(f, parsed) - case parserTypes.SumFieldName: + case request.SumFieldName: plan, aggregateError = n.p.Sum(f, parsed) - case parserTypes.AverageFieldName: + case request.AverageFieldName: plan, aggregateError = n.p.Average(f) } @@ -279,7 +280,7 @@ func (n *selectNode) initFields(parsed *mapper.Select) ([]aggregateNode, error) aggregates = append(aggregates, plan) } case *mapper.Select: - if f.Name == parserTypes.VersionFieldName { // reserved sub type for object queries + if f.Name == request.VersionFieldName { // reserved sub type for object queries commitSlct := &mapper.CommitSelect{ Select: *f, } @@ -292,7 +293,7 @@ func (n *selectNode) initFields(parsed *mapper.Select) ([]aggregateNode, error) // of that Target version we are querying. // So instead of a LatestCommit subquery, we need // a OneCommit subquery, with the supplied parameters. - commitSlct.DocKey = parsed.DocKeys.Value[0] // @todo check length + commitSlct.DocKey = client.Some(parsed.DocKeys.Value()[0]) // @todo check length commitSlct.Cid = parsed.Cid } @@ -301,7 +302,7 @@ func (n *selectNode) initFields(parsed *mapper.Select) ([]aggregateNode, error) if err := n.addSubPlan(f.Index, commitPlan); err != nil { return nil, err } - } else if f.Name == parserTypes.GroupFieldName { + } else if f.Name == request.GroupFieldName { if parsed.GroupBy == nil { return nil, errors.New("_group may only be referenced when within a groupBy query") } diff --git a/planner/sum.go b/planner/sum.go index f8a9024fb1..aa81fec7d1 100644 --- a/planner/sum.go +++ b/planner/sum.go @@ -14,11 +14,11 @@ import ( "fmt" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/core/enumerable" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) type sumNode struct { @@ -66,7 +66,7 @@ func (p *Planner) isValueFloat( ) (bool, error) { // It is important that averages are floats even if their underlying values are ints // else sum will round them down to the nearest whole number - if source.ChildTarget.Name == parserTypes.AverageFieldName { + if source.ChildTarget.Name == request.AverageFieldName { return true, nil } @@ -89,7 +89,7 @@ func (p *Planner) isValueFloat( } // If path length is two, we are summing a group or a child relationship - if source.ChildTarget.Name == parserTypes.CountFieldName { + if source.ChildTarget.Name == request.CountFieldName { // If we are summing a count, we know it is an int and can return false early return false, nil } @@ -99,7 +99,7 @@ func (p *Planner) isValueFloat( return false, errors.New("expected child select but none was found") } - if _, isAggregate := parserTypes.Aggregates[source.ChildTarget.Name]; isAggregate { + if _, isAggregate := request.Aggregates[source.ChildTarget.Name]; isAggregate { // If we are aggregating an aggregate, we need to traverse the aggregation chain down to // the root field in order to determine the value type. This is recursive to allow handling // of N-depth aggregations (e.g. sum of sum of sum of...) diff --git a/planner/top.go b/planner/top.go index 21e8a84b9c..3ea6971129 100644 --- a/planner/top.go +++ b/planner/top.go @@ -11,10 +11,10 @@ package planner import ( + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/errors" "github.com/sourcenetwork/defradb/mapper" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) const topLevelNodeKind string = "topLevelNode" @@ -189,11 +189,11 @@ func (p *Planner) Top(m *mapper.Select) (*topLevelNode, error) { var child planNode var err error switch field.GetName() { - case parserTypes.CountFieldName: + case request.CountFieldName: child, err = p.Count(f, m) - case parserTypes.SumFieldName: + case request.SumFieldName: child, err = p.Sum(f, m) - case parserTypes.AverageFieldName: + case request.AverageFieldName: child, err = p.Average(f) } if err != nil { diff --git a/planner/update.go b/planner/update.go index 7ce0f120d3..23cea4bd82 100644 --- a/planner/update.go +++ b/planner/update.go @@ -126,7 +126,7 @@ func (p *Planner) UpdateDocs(parsed *mapper.Mutation) (planNode, error) { update := &updateNode{ p: p, filter: parsed.Filter, - ids: parsed.DocKeys.Value, + ids: parsed.DocKeys.Value(), isUpdating: true, patch: parsed.Data, docMapper: docMapper{&parsed.DocumentMapping}, diff --git a/query/graphql/parser/commit.go b/query/graphql/parser/commit.go index cdb1dcf8c8..f40c5828b0 100644 --- a/query/graphql/parser/commit.go +++ b/query/graphql/parser/commit.go @@ -16,114 +16,78 @@ import ( "github.com/graphql-go/graphql/language/ast" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/core" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) -var ( - _ Selection = (*CommitSelect)(nil) -) - -type CommitSelect struct { - Alias string - Name string - - DocKey string - FieldName client.Option[string] - Cid client.Option[string] - Depth client.Option[uint64] - - Limit *parserTypes.Limit - OrderBy *parserTypes.OrderBy - GroupBy *parserTypes.GroupBy - - Fields []Selection -} - -func (c CommitSelect) GetRoot() parserTypes.SelectionType { - return parserTypes.CommitSelection -} - -func (c CommitSelect) ToSelect() *Select { - return &Select{ - Alias: c.Alias, - Limit: c.Limit, - OrderBy: c.OrderBy, - GroupBy: c.GroupBy, - Fields: c.Fields, - Root: parserTypes.CommitSelection, - } -} - -func parseCommitSelect(field *ast.Field) (*CommitSelect, error) { - commit := &CommitSelect{ - Name: field.Name.Value, - Alias: getFieldAlias(field), +func parseCommitSelect(field *ast.Field) (*request.CommitSelect, error) { + commit := &request.CommitSelect{ + Field: request.Field{ + Name: field.Name.Value, + Alias: getFieldAlias(field), + }, } for _, argument := range field.Arguments { prop := argument.Name.Value - if prop == parserTypes.DocKey { + if prop == request.DocKey { raw := argument.Value.(*ast.StringValue) - commit.DocKey = raw.Value - } else if prop == parserTypes.Cid { + commit.DocKey = client.Some(raw.Value) + } else if prop == request.Cid { raw := argument.Value.(*ast.StringValue) commit.Cid = client.Some(raw.Value) - } else if prop == parserTypes.Field { + } else if prop == request.FieldName { raw := argument.Value.(*ast.StringValue) commit.FieldName = client.Some(raw.Value) - } else if prop == parserTypes.OrderClause { + } else if prop == request.OrderClause { obj := argument.Value.(*ast.ObjectValue) cond, err := ParseConditionsInOrder(obj) if err != nil { return nil, err } - commit.OrderBy = &parserTypes.OrderBy{ - Conditions: cond, - Statement: obj, - } - } else if prop == parserTypes.LimitClause { + commit.OrderBy = client.Some( + request.OrderBy{ + Conditions: cond, + }, + ) + } else if prop == request.LimitClause { val := argument.Value.(*ast.IntValue) - limit, err := strconv.ParseInt(val.Value, 10, 64) + limit, err := strconv.ParseUint(val.Value, 10, 64) if err != nil { return nil, err } - if commit.Limit == nil { - commit.Limit = &parserTypes.Limit{} - } - commit.Limit.Limit = limit - } else if prop == parserTypes.OffsetClause { + commit.Limit = client.Some(limit) + } else if prop == request.OffsetClause { val := argument.Value.(*ast.IntValue) - offset, err := strconv.ParseInt(val.Value, 10, 64) + offset, err := strconv.ParseUint(val.Value, 10, 64) if err != nil { return nil, err } - if commit.Limit == nil { - commit.Limit = &parserTypes.Limit{} - } - commit.Limit.Offset = offset - } else if prop == parserTypes.DepthClause { + commit.Offset = client.Some(offset) + } else if prop == request.DepthClause { raw := argument.Value.(*ast.IntValue) depth, err := strconv.ParseUint(raw.Value, 10, 64) if err != nil { return nil, err } commit.Depth = client.Some(depth) - } else if prop == parserTypes.GroupByClause { + } else if prop == request.GroupByClause { obj := argument.Value.(*ast.ListValue) fields := []string{} for _, v := range obj.Values { fields = append(fields, v.GetValue().(string)) } - commit.GroupBy = &parserTypes.GroupBy{ - Fields: fields, - } + commit.GroupBy = client.Some( + request.GroupBy{ + Fields: fields, + }, + ) } } // latestCommits is just syntax sugar around a commits query - if commit.Name == parserTypes.LatestCommitsQueryName { + if commit.Name == request.LatestCommitsQueryName { // Depth is not exposed as an input parameter for latestCommits, // so we can blindly set it here without worrying about existing // values @@ -141,7 +105,7 @@ func parseCommitSelect(field *ast.Field) (*CommitSelect, error) { } var err error - commit.Fields, err = parseSelectFields(commit.GetRoot(), field.SelectionSet) + commit.Fields, err = parseSelectFields(request.CommitSelection, field.SelectionSet) return commit, err } diff --git a/query/graphql/parser/filter.go b/query/graphql/parser/filter.go index 7cd7c156fc..37ea6399bb 100644 --- a/query/graphql/parser/filter.go +++ b/query/graphql/parser/filter.go @@ -11,7 +11,6 @@ package parser import ( - "fmt" "strconv" "strings" @@ -19,46 +18,38 @@ import ( gqlp "github.com/graphql-go/graphql/language/parser" gqls "github.com/graphql-go/graphql/language/source" + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/errors" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) -// Filter contains the parsed condition map to be -// run by the Filter Evaluator. -// @todo: Cache filter structure for faster condition -// evaluation. -type Filter struct { - // parsed filter conditions - Conditions map[string]any -} - // type condition // NewFilter parses the given GraphQL ObjectValue AST type // and extracts all the filter conditions into a usable map. -func NewFilter(stmt *ast.ObjectValue) (*Filter, error) { +func NewFilter(stmt *ast.ObjectValue) (client.Option[request.Filter], error) { conditions, err := ParseConditions(stmt) if err != nil { - return nil, err + return client.None[request.Filter](), err } - return &Filter{ + return client.Some(request.Filter{ Conditions: conditions, - }, nil + }), nil } // NewFilterFromString creates a new filter from a string. -func NewFilterFromString(body string) (*Filter, error) { +func NewFilterFromString(body string) (client.Option[request.Filter], error) { if !strings.HasPrefix(body, "{") { body = "{" + body + "}" } src := gqls.NewSource(&gqls.Source{Body: []byte(body)}) p, err := gqlp.MakeParser(src, gqlp.ParseOptions{}) if err != nil { - return nil, err + return client.None[request.Filter](), err } obj, err := gqlp.ParseObject(p, false) if err != nil { - return nil, err + return client.None[request.Filter](), err } return NewFilter(obj) @@ -72,20 +63,20 @@ type parseFn func(*ast.ObjectValue) (any, error) // This function is mostly used by the Order parser, which needs to parse // conditions in the same way as the Filter object, however the order // of the arguments is important. -func ParseConditionsInOrder(stmt *ast.ObjectValue) ([]parserTypes.OrderCondition, error) { +func ParseConditionsInOrder(stmt *ast.ObjectValue) ([]request.OrderCondition, error) { cond, err := parseConditionsInOrder(stmt) if err != nil { return nil, err } - if v, ok := cond.([]parserTypes.OrderCondition); ok { + if v, ok := cond.([]request.OrderCondition); ok { return v, nil } return nil, errors.New("failed to parse statement") } func parseConditionsInOrder(stmt *ast.ObjectValue) (any, error) { - conditions := make([]parserTypes.OrderCondition, 0) + conditions := make([]request.OrderCondition, 0) if stmt == nil { return conditions, nil } @@ -98,23 +89,23 @@ func parseConditionsInOrder(stmt *ast.ObjectValue) (any, error) { switch v := val.(type) { case string: // base direction parsed (hopefully, check NameToOrderDirection) - dir, ok := parserTypes.NameToOrderDirection[v] + dir, ok := request.NameToOrderDirection[v] if !ok { return nil, errors.New("invalid order direction string") } - conditions = append(conditions, parserTypes.OrderCondition{ - Field: name, + conditions = append(conditions, request.OrderCondition{ + Fields: []string{name}, Direction: dir, }) - case []parserTypes.OrderCondition: // flatten and incorporate the parsed slice into our current one + case []request.OrderCondition: // flatten and incorporate the parsed slice into our current one for _, cond := range v { // prepend the current field name, to the parsed condition from the slice // Eg. order: {author: {name: ASC, birthday: DESC}} // This results in an array of [name, birthday] converted to // [author.name, author.birthday]. // etc. - cond.Field = fmt.Sprintf("%s.%s", name, cond.Field) + cond.Fields = append([]string{name}, cond.Fields...) conditions = append(conditions, cond) } diff --git a/query/graphql/parser/mutation.go b/query/graphql/parser/mutation.go index eca2fc3f21..17cf9e498f 100644 --- a/query/graphql/parser/mutation.go +++ b/query/graphql/parser/mutation.go @@ -15,84 +15,33 @@ import ( "github.com/graphql-go/graphql/language/ast" + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/errors" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" -) - -type MutationType int - -const ( - NoneMutationType = MutationType(iota) - CreateObjects - UpdateObjects - DeleteObjects -) - -var ( - mutationNameToType = map[string]MutationType{ - "create": CreateObjects, - "update": UpdateObjects, - "delete": DeleteObjects, - } ) var ( ErrEmptyDataPayload = errors.New("given data payload is empty") ) -// Mutation is a field on the MutationType -// of a graphql query. It includes all the possible -// arguments and all -// -// @todo: Change name to ObjectMutation to indicate -// generated object mutation actions -type Mutation struct { - Name string - Alias string - Type MutationType - - // Schema is the target schema/collection - // if this mutation is on an object. - Schema string - - IDs parserTypes.OptionalDocKeys - Filter *Filter - Data string - - Fields []Selection -} - -func (m Mutation) GetRoot() parserTypes.SelectionType { - return parserTypes.ObjectSelection -} - -// ToSelect returns a basic Select object, with the same Name, Alias, and Fields as -// the Mutation object. Used to create a Select planNode for the mutation return objects. -func (m Mutation) ToSelect() *Select { - return &Select{ - Name: m.Schema, - Alias: m.Alias, - Fields: m.Fields, - DocKeys: m.IDs, - Filter: m.Filter, +var ( + mutationNameToType = map[string]request.MutationType{ + "create": request.CreateObjects, + "update": request.UpdateObjects, + "delete": request.DeleteObjects, } -} +) // parseMutationOperationDefinition parses the individual GraphQL // 'mutation' operations, which there may be multiple of. -func parseMutationOperationDefinition(def *ast.OperationDefinition) (*OperationDefinition, error) { - qdef := &OperationDefinition{ - Statement: def, - Selections: make([]Selection, len(def.SelectionSet.Selections)), - } - - if def.Name != nil { - qdef.Name = def.Name.Value +func parseMutationOperationDefinition(def *ast.OperationDefinition) (*request.OperationDefinition, error) { + qdef := &request.OperationDefinition{ + Selections: make([]request.Selection, len(def.SelectionSet.Selections)), } qdef.IsExplain = parseExplainDirective(def.Directives) - for i, selection := range qdef.Statement.SelectionSet.Selections { + for i, selection := range def.SelectionSet.Selections { switch node := selection.(type) { case *ast.Field: mut, err := parseMutation(node) @@ -113,11 +62,12 @@ func parseMutationOperationDefinition(def *ast.OperationDefinition) (*OperationD // parseMutation parses a typed mutation field // which includes sub fields, and may include // filters, IDs, payloads, etc. -func parseMutation(field *ast.Field) (*Mutation, error) { - mut := &Mutation{} - mut.Name = field.Name.Value - if field.Alias != nil { - mut.Alias = field.Alias.Value +func parseMutation(field *ast.Field) (*request.Mutation, error) { + mut := &request.Mutation{ + Field: request.Field{ + Name: field.Name.Value, + Alias: getFieldAlias(field), + }, } // parse the mutation type @@ -149,13 +99,13 @@ func parseMutation(field *ast.Field) (*Mutation, error) { for _, argument := range field.Arguments { prop := argument.Name.Value // parse each individual arg type seperately - if prop == parserTypes.Data { // parse data + if prop == request.Data { // parse data raw := argument.Value.(*ast.StringValue) if raw.Value == "" { return nil, ErrEmptyDataPayload } mut.Data = raw.Value - } else if prop == parserTypes.FilterClause { // parse filter + } else if prop == request.FilterClause { // parse filter obj := argument.Value.(*ast.ObjectValue) filter, err := NewFilter(obj) if err != nil { @@ -163,13 +113,10 @@ func parseMutation(field *ast.Field) (*Mutation, error) { } mut.Filter = filter - } else if prop == parserTypes.Id { + } else if prop == request.Id { raw := argument.Value.(*ast.StringValue) - mut.IDs = parserTypes.OptionalDocKeys{ - HasValue: true, - Value: []string{raw.Value}, - } - } else if prop == parserTypes.Ids { + mut.IDs = client.Some([]string{raw.Value}) + } else if prop == request.Ids { raw := argument.Value.(*ast.ListValue) ids := make([]string, len(raw.Values)) for i, val := range raw.Values { @@ -179,10 +126,7 @@ func parseMutation(field *ast.Field) (*Mutation, error) { } ids[i] = id.Value } - mut.IDs = parserTypes.OptionalDocKeys{ - HasValue: true, - Value: ids, - } + mut.IDs = client.Some(ids) } } @@ -192,6 +136,6 @@ func parseMutation(field *ast.Field) (*Mutation, error) { } var err error - mut.Fields, err = parseSelectFields(mut.GetRoot(), field.SelectionSet) + mut.Fields, err = parseSelectFields(request.ObjectSelection, field.SelectionSet) return mut, err } diff --git a/query/graphql/parser/query.go b/query/graphql/parser/query.go index 3bfc760be7..2c1d603f63 100644 --- a/query/graphql/parser/query.go +++ b/query/graphql/parser/query.go @@ -16,159 +16,23 @@ import ( "github.com/graphql-go/graphql/language/ast" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/errors" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) -var dbAPIQueryNames = map[string]bool{ - "latestCommits": true, - "commits": true, - "commit": true, -} - -type Query struct { - Queries []*OperationDefinition - Mutations []*OperationDefinition - Statement *ast.Document -} - -type OperationDefinition struct { - Name string - Selections []Selection - Statement *ast.OperationDefinition - IsExplain bool -} - -func (q OperationDefinition) GetStatement() ast.Node { - return q.Statement -} - -type Selection interface { - GetRoot() parserTypes.SelectionType -} - -// Select is a complex Field with strong typing -// It used for sub types in a query. Includes -// fields, and query arguments like filters, -// limits, etc. -type Select struct { - Name string - // The identifier to be used in the rendered results, typically specified by - // the user. - Alias string - - DocKeys parserTypes.OptionalDocKeys - CID client.Option[string] - - // QueryType indicates what kind of query this is - // Currently supports: ScanQuery, VersionedScanQuery - QueryType parserTypes.SelectQueryType - - // Root is the top level query parsed type - Root parserTypes.SelectionType - - Limit *parserTypes.Limit - - OrderBy *parserTypes.OrderBy - - GroupBy *parserTypes.GroupBy - - Filter *Filter - - Fields []Selection - - // Raw graphql statement - Statement *ast.Field -} - -func (s Select) GetRoot() parserTypes.SelectionType { - return s.Root -} - -func (s Select) validate() []error { - result := []error{} - - result = append(result, s.validateShallow()...) - - for _, childSelection := range s.Fields { - switch typedChildSelection := childSelection.(type) { - case *Select: - result = append(result, typedChildSelection.validateShallow()...) - default: - // Do nothing - } - } - - return result -} - -func (s Select) validateShallow() []error { - result := []error{} - - result = append(result, s.validateGroupBy()...) - - return result -} - -func (s Select) validateGroupBy() []error { - result := []error{} - - if s.GroupBy == nil { - return result - } - - for _, childSelection := range s.Fields { - switch typedChildSelection := childSelection.(type) { - case *Field: - if typedChildSelection.Name == parserTypes.TypeNameFieldName { - // _typeName is permitted - continue - } - - var fieldExistsInGroupBy bool - for _, groupByField := range s.GroupBy.Fields { - if typedChildSelection.Name == groupByField { - fieldExistsInGroupBy = true - break - } - } - if !fieldExistsInGroupBy { - result = append(result, client.NewErrSelectOfNonGroupField(typedChildSelection.Name)) - } - default: - // Do nothing - } - } - - return result -} - -// Field implements Selection -type Field struct { - Name string - Alias string - - Root parserTypes.SelectionType -} - -func (c Field) GetRoot() parserTypes.SelectionType { - return c.Root -} - // ParseQuery parses a root ast.Document, and returns a // formatted Query object. // Requires a non-nil doc, will error if given a nil doc. -func ParseQuery(doc *ast.Document) (*Query, []error) { +func ParseQuery(doc *ast.Document) (*request.Request, []error) { if doc == nil { return nil, []error{errors.New("parseQuery requires a non-nil ast.Document")} } - q := &Query{ - Statement: doc, - Queries: make([]*OperationDefinition, 0), - Mutations: make([]*OperationDefinition, 0), + r := &request.Request{ + Queries: make([]*request.OperationDefinition, 0), + Mutations: make([]*request.OperationDefinition, 0), } - for _, def := range q.Statement.Definitions { + for _, def := range doc.Definitions { switch node := def.(type) { case *ast.OperationDefinition: if node.Operation == "query" { @@ -177,21 +41,21 @@ func ParseQuery(doc *ast.Document) (*Query, []error) { if err != nil { return nil, err } - q.Queries = append(q.Queries, qdef) + r.Queries = append(r.Queries, qdef) } else if node.Operation == "mutation" { // parse mutation operation definition. mdef, err := parseMutationOperationDefinition(node) if err != nil { return nil, []error{err} } - q.Mutations = append(q.Mutations, mdef) + r.Mutations = append(r.Mutations, mdef) } else { return nil, []error{errors.New("unknown GraphQL operation type")} } } } - return q, nil + return r, nil } // parseExplainDirective returns true if we parsed / detected the explain directive label @@ -203,7 +67,7 @@ func parseExplainDirective(directives []*ast.Directive) bool { // unless we add another directive named `@explain` at another location (which we should not). for _, directive := range directives { // The arguments pased to the directive are at `directive.Arguments`. - if directive.Name.Value == parserTypes.ExplainLabel { + if directive.Name.Value == request.ExplainLabel { return true } } @@ -213,41 +77,49 @@ func parseExplainDirective(directives []*ast.Directive) bool { // parseQueryOperationDefinition parses the individual GraphQL // 'query' operations, which there may be multiple of. -func parseQueryOperationDefinition(def *ast.OperationDefinition) (*OperationDefinition, []error) { - qdef := &OperationDefinition{ - Statement: def, - Selections: make([]Selection, len(def.SelectionSet.Selections)), - } - - if def.Name != nil { - qdef.Name = def.Name.Value +func parseQueryOperationDefinition(def *ast.OperationDefinition) (*request.OperationDefinition, []error) { + qdef := &request.OperationDefinition{ + Selections: make([]request.Selection, len(def.SelectionSet.Selections)), } qdef.IsExplain = parseExplainDirective(def.Directives) - for i, selection := range qdef.Statement.SelectionSet.Selections { - var parsedSelection Selection + for i, selection := range def.SelectionSet.Selections { + var parsedSelection request.Selection switch node := selection.(type) { case *ast.Field: - // which query type is this database API query object query etc. - _, exists := dbAPIQueryNames[node.Name.Value] - if exists { - // the query matches a reserved DB API query name - parsed, err := parseAPIQuery(node) + if _, isCommitQuery := request.CommitQueries[node.Name.Value]; isCommitQuery { + parsed, err := parseCommitSelect(node) if err != nil { return nil, []error{err} } parsedSelection = parsed + } else if _, isAggregate := request.Aggregates[node.Name.Value]; isAggregate { + parsed, err := parseAggregate(node, i) + if err != nil { + return nil, []error{err} + } + + // Top-level aggregates must be wrapped in a top-level Select for now + parsedSelection = &request.Select{ + Field: request.Field{ + Name: parsed.Name, + Alias: parsed.Alias, + }, + Fields: []request.Selection{ + parsed, + }, + } } else { // the query doesn't match a reserve name // so its probably a generated query - parsed, err := parseSelect(parserTypes.ObjectSelection, node, i) + parsed, err := parseSelect(request.ObjectSelection, node, i) if err != nil { return nil, []error{err} } - errors := parsed.validate() + errors := parsed.Validate() if len(errors) > 0 { return nil, errors } @@ -268,12 +140,13 @@ func parseQueryOperationDefinition(def *ast.OperationDefinition) (*OperationDefi // parseSelect parses a typed selection field // which includes sub fields, and may include // filters, limits, orders, etc.. -func parseSelect(rootType parserTypes.SelectionType, field *ast.Field, index int) (*Select, error) { - slct := &Select{ - Alias: getFieldAlias(field), - Name: field.Name.Value, - Root: rootType, - Statement: field, +func parseSelect(rootType request.SelectionType, field *ast.Field, index int) (*request.Select, error) { + slct := &request.Select{ + Field: request.Field{ + Name: field.Name.Value, + Alias: getFieldAlias(field), + }, + Root: rootType, } // parse arguments @@ -282,7 +155,7 @@ func parseSelect(rootType parserTypes.SelectionType, field *ast.Field, index int astValue := argument.Value // parse filter - if prop == parserTypes.FilterClause { + if prop == request.FilterClause { obj := astValue.(*ast.ObjectValue) filter, err := NewFilter(obj) if err != nil { @@ -290,71 +163,56 @@ func parseSelect(rootType parserTypes.SelectionType, field *ast.Field, index int } slct.Filter = filter - } else if prop == parserTypes.DocKey { // parse single dockey query field + } else if prop == request.DocKey { // parse single dockey query field val := astValue.(*ast.StringValue) - slct.DocKeys = parserTypes.OptionalDocKeys{ - HasValue: true, - Value: []string{val.Value}, - } - } else if prop == parserTypes.DocKeys { + slct.DocKeys = client.Some([]string{val.Value}) + } else if prop == request.DocKeys { docKeyValues := astValue.(*ast.ListValue).Values docKeys := make([]string, len(docKeyValues)) for i, value := range docKeyValues { docKeys[i] = value.(*ast.StringValue).Value } - slct.DocKeys = parserTypes.OptionalDocKeys{ - HasValue: true, - Value: docKeys, - } - } else if prop == parserTypes.Cid { // parse single CID query field + slct.DocKeys = client.Some(docKeys) + } else if prop == request.Cid { // parse single CID query field val := astValue.(*ast.StringValue) slct.CID = client.Some(val.Value) - } else if prop == parserTypes.LimitClause { // parse limit/offset + } else if prop == request.LimitClause { // parse limit/offset val := astValue.(*ast.IntValue) - i, err := strconv.ParseInt(val.Value, 10, 64) + limit, err := strconv.ParseUint(val.Value, 10, 64) if err != nil { - return slct, err - } - if slct.Limit == nil { - slct.Limit = &parserTypes.Limit{} + return nil, err } - slct.Limit.Limit = i - } else if prop == parserTypes.OffsetClause { // parse limit/offset + slct.Limit = client.Some(limit) + } else if prop == request.OffsetClause { // parse limit/offset val := astValue.(*ast.IntValue) - i, err := strconv.ParseInt(val.Value, 10, 64) + offset, err := strconv.ParseUint(val.Value, 10, 64) if err != nil { - return slct, err - } - if slct.Limit == nil { - slct.Limit = &parserTypes.Limit{} + return nil, err } - slct.Limit.Offset = i - } else if prop == parserTypes.OrderClause { // parse order by + slct.Offset = client.Some(offset) + } else if prop == request.OrderClause { // parse order by obj := astValue.(*ast.ObjectValue) cond, err := ParseConditionsInOrder(obj) if err != nil { return nil, err } - slct.OrderBy = &parserTypes.OrderBy{ - Conditions: cond, - Statement: obj, - } - } else if prop == parserTypes.GroupByClause { + slct.OrderBy = client.Some( + request.OrderBy{ + Conditions: cond, + }, + ) + } else if prop == request.GroupByClause { obj := astValue.(*ast.ListValue) fields := make([]string, 0) for _, v := range obj.Values { fields = append(fields, v.GetValue().(string)) } - slct.GroupBy = &parserTypes.GroupBy{ - Fields: fields, - } - } - - if len(slct.DocKeys.Value) != 0 && slct.CID.HasValue() { - slct.QueryType = parserTypes.VersionedScanQuery - } else { - slct.QueryType = parserTypes.ScanQuery + slct.GroupBy = client.Some( + request.GroupBy{ + Fields: fields, + }, + ) } } @@ -373,32 +231,32 @@ func parseSelect(rootType parserTypes.SelectionType, field *ast.Field, index int return slct, err } -func getFieldAlias(field *ast.Field) string { +func getFieldAlias(field *ast.Field) client.Option[string] { if field.Alias == nil { - return field.Name.Value + return client.None[string]() } - return field.Alias.Value + return client.Some(field.Alias.Value) } -func parseSelectFields(root parserTypes.SelectionType, fields *ast.SelectionSet) ([]Selection, error) { - selections := make([]Selection, len(fields.Selections)) +func parseSelectFields(root request.SelectionType, fields *ast.SelectionSet) ([]request.Selection, error) { + selections := make([]request.Selection, len(fields.Selections)) // parse field selections for i, selection := range fields.Selections { switch node := selection.(type) { case *ast.Field: - if _, isAggregate := parserTypes.Aggregates[node.Name.Value]; isAggregate { - s, err := parseSelect(root, node, i) + if _, isAggregate := request.Aggregates[node.Name.Value]; isAggregate { + s, err := parseAggregate(node, i) if err != nil { return nil, err } selections[i] = s } else if node.SelectionSet == nil { // regular field - selections[i] = parseField(root, node) + selections[i] = parseField(node) } else { // sub type with extra fields subroot := root switch node.Name.Value { - case parserTypes.VersionFieldName: - subroot = parserTypes.CommitSelection + case request.VersionFieldName: + subroot = request.CommitSelection } s, err := parseSelect(subroot, node, i) if err != nil { @@ -414,19 +272,125 @@ func parseSelectFields(root parserTypes.SelectionType, fields *ast.SelectionSet) // parseField simply parses the Name/Alias // into a Field type -func parseField(root parserTypes.SelectionType, field *ast.Field) *Field { - return &Field{ - Root: root, +func parseField(field *ast.Field) *request.Field { + return &request.Field{ Name: field.Name.Value, Alias: getFieldAlias(field), } } -func parseAPIQuery(field *ast.Field) (Selection, error) { - switch field.Name.Value { - case "latestCommits", "commits": - return parseCommitSelect(field) - default: - return nil, errors.New("unknown query") +func parseAggregate(field *ast.Field, index int) (*request.Aggregate, error) { + targets := make([]*request.AggregateTarget, len(field.Arguments)) + + for i, argument := range field.Arguments { + switch argumentValue := argument.Value.GetValue().(type) { + case string: + targets[i] = &request.AggregateTarget{ + HostName: argumentValue, + } + case []*ast.ObjectField: + hostName := argument.Name.Value + var childName string + var filter client.Option[request.Filter] + var limit client.Option[uint64] + var offset client.Option[uint64] + var order client.Option[request.OrderBy] + + fieldArg, hasFieldArg := tryGet(argumentValue, request.FieldName) + if hasFieldArg { + if innerPathStringValue, isString := fieldArg.Value.GetValue().(string); isString { + childName = innerPathStringValue + } + } + + filterArg, hasFilterArg := tryGet(argumentValue, request.FilterClause) + if hasFilterArg { + filterValue, err := NewFilter(filterArg.Value.(*ast.ObjectValue)) + if err != nil { + return nil, err + } + filter = filterValue + } + + limitArg, hasLimitArg := tryGet(argumentValue, request.LimitClause) + if hasLimitArg { + limitValue, err := strconv.ParseUint(limitArg.Value.(*ast.IntValue).Value, 10, 64) + if err != nil { + return nil, err + } + limit = client.Some(limitValue) + } + + offsetArg, hasOffsetArg := tryGet(argumentValue, request.OffsetClause) + if hasOffsetArg { + offsetValue, err := strconv.ParseUint(offsetArg.Value.(*ast.IntValue).Value, 10, 64) + if err != nil { + return nil, err + } + offset = client.Some(offsetValue) + } + + orderArg, hasOrderArg := tryGet(argumentValue, request.OrderClause) + if hasOrderArg { + switch orderArgValue := orderArg.Value.(type) { + case *ast.EnumValue: + // For inline arrays the order arg will be a simple enum declaring the order direction + orderDirectionString := orderArgValue.Value + orderDirection := request.OrderDirection(orderDirectionString) + + order = client.Some( + request.OrderBy{ + Conditions: []request.OrderCondition{ + { + Direction: orderDirection, + }, + }, + }, + ) + + case *ast.ObjectValue: + // For relations the order arg will be the complex order object as used by the host object + // for non-aggregate ordering + + // We use the parser package parsing for convienience here + orderConditions, err := ParseConditionsInOrder(orderArgValue) + if err != nil { + return nil, err + } + + order = client.Some( + request.OrderBy{ + Conditions: orderConditions, + }, + ) + } + } + + targets[i] = &request.AggregateTarget{ + HostName: hostName, + ChildName: client.Some(childName), + Filter: filter, + Limit: limit, + Offset: offset, + OrderBy: order, + } + } + } + + return &request.Aggregate{ + Field: request.Field{ + Name: field.Name.Value, + Alias: getFieldAlias(field), + }, + Targets: targets, + }, nil +} + +func tryGet(fields []*ast.ObjectField, name string) (*ast.ObjectField, bool) { + for _, field := range fields { + if field.Name.Value == name { + return field, true + } } + return nil, false } diff --git a/query/graphql/schema/descriptions.go b/query/graphql/schema/descriptions.go index a92a7e440c..b4d9485285 100644 --- a/query/graphql/schema/descriptions.go +++ b/query/graphql/schema/descriptions.go @@ -18,8 +18,8 @@ import ( gql "github.com/graphql-go/graphql" "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/client/request" "github.com/sourcenetwork/defradb/errors" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" ) var ( @@ -146,7 +146,7 @@ func (g *Generator) CreateDescriptions( Name: t.Name(), Fields: []client.FieldDescription{ { - Name: parserTypes.DocKeyFieldName, + Name: request.DocKeyFieldName, Kind: client.FieldKind_DocKey, Typ: client.NONE_CRDT, }, @@ -154,7 +154,7 @@ func (g *Generator) CreateDescriptions( } // and schema fields for fname, field := range t.Fields() { - if _, ok := parserTypes.ReservedFields[fname]; ok { + if _, ok := request.ReservedFields[fname]; ok { continue } @@ -236,9 +236,9 @@ func (g *Generator) CreateDescriptions( // sort the fields lexicographically sort.Slice(desc.Schema.Fields, func(i, j int) bool { // make sure that the _key (DocKeyFieldName) is always at the beginning - if desc.Schema.Fields[i].Name == parserTypes.DocKeyFieldName { + if desc.Schema.Fields[i].Name == request.DocKeyFieldName { return true - } else if desc.Schema.Fields[j].Name == parserTypes.DocKeyFieldName { + } else if desc.Schema.Fields[j].Name == request.DocKeyFieldName { return false } return desc.Schema.Fields[i].Name < desc.Schema.Fields[j].Name diff --git a/query/graphql/schema/generate.go b/query/graphql/schema/generate.go index 25bb2e485b..817bccf981 100644 --- a/query/graphql/schema/generate.go +++ b/query/graphql/schema/generate.go @@ -22,7 +22,7 @@ import ( "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/errors" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" + "github.com/sourcenetwork/defradb/client/request" schemaTypes "github.com/sourcenetwork/defradb/query/graphql/schema/types" ) @@ -174,12 +174,12 @@ func (g *Generator) fromAST(ctx context.Context, document *ast.Document) ([]*gql return nil, err } case *gql.Scalar: - if _, isAggregate := parserTypes.Aggregates[def.Name]; isAggregate { + if _, isAggregate := request.Aggregates[def.Name]; isAggregate { for name, aggregateTarget := range def.Args { expandedField := &gql.InputObjectFieldConfig{ Type: g.manager.schema.TypeMap()[name+"FilterArg"], } - aggregateTarget.Type.(*gql.InputObject).AddFieldConfig(parserTypes.FilterClause, expandedField) + aggregateTarget.Type.(*gql.InputObject).AddFieldConfig(request.FilterClause, expandedField) } } } @@ -217,8 +217,8 @@ func (g *Generator) expandInputArgument(obj *gql.Object) error { fields := obj.Fields() for f, def := range fields { // ignore reserved fields, execpt the Group field (as that requires typing), and aggregates - if _, ok := parserTypes.ReservedFields[f]; ok && f != parserTypes.GroupFieldName { - if _, isAggregate := parserTypes.Aggregates[f]; !isAggregate { + if _, ok := request.ReservedFields[f]; ok && f != request.GroupFieldName { + if _, isAggregate := request.Aggregates[f]; !isAggregate { continue } } @@ -264,7 +264,7 @@ func (g *Generator) expandInputArgument(obj *gql.Object) error { obj.AddFieldConfig(f, expandedField) } case *gql.Scalar: - if _, isAggregate := parserTypes.Aggregates[f]; isAggregate { + if _, isAggregate := request.Aggregates[f]; isAggregate { if err := g.createExpandedFieldAggregate(obj, def); err != nil { return err } @@ -285,7 +285,7 @@ func (g *Generator) createExpandedFieldAggregate( for _, aggregateTarget := range f.Args { target := aggregateTarget.Name() var filterTypeName string - if target == parserTypes.GroupFieldName { + if target == request.GroupFieldName { filterTypeName = obj.Name() + "FilterArg" } else { if targeted := obj.Fields()[target]; targeted != nil { @@ -356,9 +356,9 @@ func (g *Generator) createExpandedFieldList( "groupBy": schemaTypes.NewArgConfig( gql.NewList(gql.NewNonNull(g.manager.schema.TypeMap()[typeName+"Fields"])), ), - "order": schemaTypes.NewArgConfig(g.manager.schema.TypeMap()[typeName+"OrderArg"]), - parserTypes.LimitClause: schemaTypes.NewArgConfig(gql.Int), - parserTypes.OffsetClause: schemaTypes.NewArgConfig(gql.Int), + "order": schemaTypes.NewArgConfig(g.manager.schema.TypeMap()[typeName+"OrderArg"]), + request.LimitClause: schemaTypes.NewArgConfig(gql.Int), + request.OffsetClause: schemaTypes.NewArgConfig(gql.Int), }, } @@ -407,7 +407,7 @@ func (g *Generator) buildTypesFromAST( // @todo: Check if this is a collection (relation) type // or just a embedded only type (which doesn't need a key) // automatically add the _key: ID field to the type - fields[parserTypes.DocKeyFieldName] = &gql.Field{Type: gql.ID} + fields[request.DocKeyFieldName] = &gql.Field{Type: gql.ID} for _, field := range defType.Fields { fType := new(gql.Field) @@ -493,7 +493,7 @@ func (g *Generator) buildTypesFromAST( )) } - fields[parserTypes.GroupFieldName] = &gql.Field{ + fields[request.GroupFieldName] = &gql.Field{ Type: gql.NewList(gqlType), } @@ -616,7 +616,7 @@ func (g *Generator) genAggregateFields(ctx context.Context) error { func genTopLevelCount(topLevelCountInputs map[string]*gql.InputObject) *gql.Field { topLevelCountField := gql.Field{ - Name: parserTypes.CountFieldName, + Name: request.CountFieldName, Type: gql.Int, Args: gql.FieldConfigArgument{}, } @@ -630,13 +630,13 @@ func genTopLevelCount(topLevelCountInputs map[string]*gql.InputObject) *gql.Fiel func genTopLevelNumericAggregates(topLevelNumericAggInputs map[string]*gql.InputObject) []*gql.Field { topLevelSumField := gql.Field{ - Name: parserTypes.SumFieldName, + Name: request.SumFieldName, Type: gql.Float, Args: gql.FieldConfigArgument{}, } topLevelAverageField := gql.Field{ - Name: parserTypes.AverageFieldName, + Name: request.AverageFieldName, Type: gql.Float, Args: gql.FieldConfigArgument{}, } @@ -673,7 +673,7 @@ func (g *Generator) genCountFieldConfig(obj *gql.Object) (gql.Field, error) { } field := gql.Field{ - Name: parserTypes.CountFieldName, + Name: request.CountFieldName, Type: gql.Int, Args: gql.FieldConfigArgument{}, } @@ -712,7 +712,7 @@ func (g *Generator) genSumFieldConfig(obj *gql.Object) (gql.Field, error) { } field := gql.Field{ - Name: parserTypes.SumFieldName, + Name: request.SumFieldName, Type: gql.Float, Args: gql.FieldConfigArgument{}, } @@ -751,7 +751,7 @@ func (g *Generator) genAverageFieldConfig(obj *gql.Object) (gql.Field, error) { } field := gql.Field{ - Name: parserTypes.AverageFieldName, + Name: request.AverageFieldName, Type: gql.Float, Args: gql.FieldConfigArgument{}, } @@ -778,15 +778,15 @@ func (g *Generator) genNumericInlineArraySelectorObject(obj *gql.Object) []*gql. selectorObject := gql.NewInputObject(gql.InputObjectConfig{ Name: genNumericInlineArraySelectorName(obj.Name(), field.Name), Fields: gql.InputObjectConfigFieldMap{ - parserTypes.LimitClause: &gql.InputObjectFieldConfig{ + request.LimitClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The maximum number of child items to aggregate.", }, - parserTypes.OffsetClause: &gql.InputObjectFieldConfig{ + request.OffsetClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The index from which to start aggregating items.", }, - parserTypes.OrderClause: &gql.InputObjectFieldConfig{ + request.OrderClause: &gql.InputObjectFieldConfig{ Type: g.manager.schema.TypeMap()["Ordering"], Description: "The order in which to aggregate items.", }, @@ -811,11 +811,11 @@ func (g *Generator) genCountBaseArgInputs(obj *gql.Object) *gql.InputObject { countableObject := gql.NewInputObject(gql.InputObjectConfig{ Name: genObjectCountName(obj.Name()), Fields: gql.InputObjectConfigFieldMap{ - parserTypes.LimitClause: &gql.InputObjectFieldConfig{ + request.LimitClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The maximum number of child items to count.", }, - parserTypes.OffsetClause: &gql.InputObjectFieldConfig{ + request.OffsetClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The index from which to start counting items.", }, @@ -839,11 +839,11 @@ func (g *Generator) genCountInlineArrayInputs(obj *gql.Object) []*gql.InputObjec selectorObject := gql.NewInputObject(gql.InputObjectConfig{ Name: genNumericInlineArrayCountName(obj.Name(), field.Name), Fields: gql.InputObjectConfigFieldMap{ - parserTypes.LimitClause: &gql.InputObjectFieldConfig{ + request.LimitClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The maximum number of child items to count.", }, - parserTypes.OffsetClause: &gql.InputObjectFieldConfig{ + request.OffsetClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The index from which to start counting items.", }, @@ -889,13 +889,13 @@ func (g *Generator) genNumericAggregateBaseArgInputs(obj *gql.Object) *gql.Input fieldsEnumCfg.Values[field.Name] = &gql.EnumValueConfig{Value: field.Name} } else { // If it is a related list, we need to add count in here so that we can sum it - fieldsEnumCfg.Values[parserTypes.CountFieldName] = &gql.EnumValueConfig{Value: parserTypes.CountFieldName} + fieldsEnumCfg.Values[request.CountFieldName] = &gql.EnumValueConfig{Value: request.CountFieldName} } } } // A child aggregate will always be aggregatable, as it can be present via an inner grouping - fieldsEnumCfg.Values[parserTypes.SumFieldName] = &gql.EnumValueConfig{Value: parserTypes.SumFieldName} - fieldsEnumCfg.Values[parserTypes.AverageFieldName] = &gql.EnumValueConfig{Value: parserTypes.AverageFieldName} + fieldsEnumCfg.Values[request.SumFieldName] = &gql.EnumValueConfig{Value: request.SumFieldName} + fieldsEnumCfg.Values[request.AverageFieldName] = &gql.EnumValueConfig{Value: request.AverageFieldName} if !hasSumableFields { return nil, nil @@ -913,15 +913,15 @@ func (g *Generator) genNumericAggregateBaseArgInputs(obj *gql.Object) *gql.Input "field": &gql.InputObjectFieldConfig{ Type: gql.NewNonNull(fieldsEnum), }, - parserTypes.LimitClause: &gql.InputObjectFieldConfig{ + request.LimitClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The maximum number of child items to aggregate.", }, - parserTypes.OffsetClause: &gql.InputObjectFieldConfig{ + request.OffsetClause: &gql.InputObjectFieldConfig{ Type: gql.Int, Description: "The index from which to start aggregating items.", }, - parserTypes.OrderClause: &gql.InputObjectFieldConfig{ + request.OrderClause: &gql.InputObjectFieldConfig{ Type: g.manager.schema.TypeMap()[genTypeName(obj, "OrderArg")], Description: "The order in which to aggregate items.", }, @@ -935,8 +935,8 @@ func (g *Generator) genNumericAggregateBaseArgInputs(obj *gql.Object) *gql.Input } func appendCommitChildGroupField() { - schemaTypes.CommitObject.Fields()[parserTypes.GroupFieldName] = &gql.FieldDefinition{ - Name: parserTypes.GroupFieldName, + schemaTypes.CommitObject.Fields()[request.GroupFieldName] = &gql.FieldDefinition{ + Name: request.GroupFieldName, Type: gql.NewList(schemaTypes.CommitObject), } } @@ -1125,7 +1125,7 @@ func (g *Generator) genTypeFilterArgInput(obj *gql.Object) *gql.InputObject { // generate basic filter operator blocks // @todo: Extract object field loop into its own utility func for f, field := range obj.Fields() { - if _, ok := parserTypes.ReservedFields[f]; ok && f != parserTypes.DocKeyFieldName { + if _, ok := request.ReservedFields[f]; ok && f != request.DocKeyFieldName { continue } // scalars (leafs) @@ -1228,7 +1228,7 @@ func (g *Generator) genTypeOrderArgInput(obj *gql.Object) *gql.InputObject { fields := gql.InputObjectConfigFieldMap{} for f, field := range obj.Fields() { - if _, ok := parserTypes.ReservedFields[f]; ok && f != parserTypes.DocKeyFieldName { + if _, ok := request.ReservedFields[f]; ok && f != request.DocKeyFieldName { continue } typeMap := g.manager.schema.TypeMap() @@ -1281,14 +1281,14 @@ func (g *Generator) genTypeQueryableFieldList( Name: name, Type: gql.NewList(obj), Args: gql.FieldConfigArgument{ - "dockey": schemaTypes.NewArgConfig(gql.String), - "dockeys": schemaTypes.NewArgConfig(gql.NewList(gql.NewNonNull(gql.String))), - "cid": schemaTypes.NewArgConfig(gql.String), - "filter": schemaTypes.NewArgConfig(config.filter), - "groupBy": schemaTypes.NewArgConfig(gql.NewList(gql.NewNonNull(config.groupBy))), - "order": schemaTypes.NewArgConfig(config.order), - parserTypes.LimitClause: schemaTypes.NewArgConfig(gql.Int), - parserTypes.OffsetClause: schemaTypes.NewArgConfig(gql.Int), + "dockey": schemaTypes.NewArgConfig(gql.String), + "dockeys": schemaTypes.NewArgConfig(gql.NewList(gql.NewNonNull(gql.String))), + "cid": schemaTypes.NewArgConfig(gql.String), + "filter": schemaTypes.NewArgConfig(config.filter), + "groupBy": schemaTypes.NewArgConfig(gql.NewList(gql.NewNonNull(config.groupBy))), + "order": schemaTypes.NewArgConfig(config.order), + request.LimitClause: schemaTypes.NewArgConfig(gql.Int), + request.OffsetClause: schemaTypes.NewArgConfig(gql.Int), }, } @@ -1340,7 +1340,7 @@ func findDirective(field *ast.FieldDefinition, directiveName string) (*ast.Direc typeDefs := ` ... ` -ast, err := parserTypes.Parse(typeDefs) +ast, err := request.Parse(typeDefs) types, err := buildTypesFromAST(ast) types, err := GenerateDBQuerySchema(ast) diff --git a/query/graphql/schema/types/commits.go b/query/graphql/schema/types/commits.go index 69f590f0d6..821e0a685d 100644 --- a/query/graphql/schema/types/commits.go +++ b/query/graphql/schema/types/commits.go @@ -13,7 +13,7 @@ package types import ( gql "github.com/graphql-go/graphql" - parserTypes "github.com/sourcenetwork/defradb/query/graphql/parser/types" + "github.com/sourcenetwork/defradb/client/request" ) var ( @@ -37,7 +37,7 @@ var ( // Any self referential type needs to be initalized // inside the init() func CommitObject = gql.NewObject(gql.ObjectConfig{ - Name: parserTypes.CommitTypeName, + Name: request.CommitTypeName, Fields: gql.Fields{ "height": &gql.Field{ Type: gql.Int, @@ -128,9 +128,9 @@ var ( ), ), ), - parserTypes.LimitClause: NewArgConfig(gql.Int), - parserTypes.OffsetClause: NewArgConfig(gql.Int), - parserTypes.DepthClause: NewArgConfig(gql.Int), + request.LimitClause: NewArgConfig(gql.Int), + request.OffsetClause: NewArgConfig(gql.Int), + request.DepthClause: NewArgConfig(gql.Int), }, } diff --git a/tests/integration/query/explain/group_with_limit_test.go b/tests/integration/query/explain/group_with_limit_test.go index 5c93baa4af..1f599c65d1 100644 --- a/tests/integration/query/explain/group_with_limit_test.go +++ b/tests/integration/query/explain/group_with_limit_test.go @@ -75,8 +75,8 @@ func TestExplainGroupByWithGroupLimitAndOffsetOnParentGroupBy(t *testing.T) { "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(1), - "offset": int64(1), + "limit": uint64(1), + "offset": uint64(1), "groupNode": dataMap{ "groupByFields": []string{"name"}, "childSelects": []dataMap{ @@ -173,8 +173,8 @@ func TestExplainGroupByWithGroupLimitAndOffsetOnChild(t *testing.T) { { "collectionName": "author", "limit": dataMap{ - "limit": int64(2), - "offset": int64(1), + "limit": uint64(2), + "offset": uint64(1), }, "docKeys": nil, "filter": nil, @@ -267,8 +267,8 @@ func TestExplainGroupByWithGroupLimitOnChildMultipleRendered(t *testing.T) { { "collectionName": "author", "limit": dataMap{ - "limit": int64(1), - "offset": int64(2), + "limit": uint64(1), + "offset": uint64(2), }, "docKeys": nil, "filter": nil, @@ -278,8 +278,8 @@ func TestExplainGroupByWithGroupLimitOnChildMultipleRendered(t *testing.T) { { "collectionName": "author", "limit": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), }, "docKeys": nil, "filter": nil, @@ -368,16 +368,16 @@ func TestExplainGroupByWithGroupLimitOnParentAndChild(t *testing.T) { "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(1), - "offset": int64(0), + "limit": uint64(1), + "offset": uint64(0), "groupNode": dataMap{ "groupByFields": []string{"name"}, "childSelects": []dataMap{ { "collectionName": "author", "limit": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), }, "orderBy": nil, "docKeys": nil, diff --git a/tests/integration/query/explain/with_limit_test.go b/tests/integration/query/explain/with_limit_test.go index 6ccb461575..c155891683 100644 --- a/tests/integration/query/explain/with_limit_test.go +++ b/tests/integration/query/explain/with_limit_test.go @@ -64,8 +64,8 @@ func TestExplainQueryWithOnlyLimitSpecified(t *testing.T) { "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -139,7 +139,7 @@ func TestExplainQueryWithOnlyOffsetSpecified(t *testing.T) { "selectTopNode": dataMap{ "limitNode": dataMap{ "limit": nil, - "offset": int64(2), + "offset": uint64(2), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -212,8 +212,8 @@ func TestExplainQueryWithBothLimitAndOffset(t *testing.T) { "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(3), - "offset": int64(1), + "limit": uint64(3), + "offset": uint64(1), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -368,8 +368,8 @@ func TestExplainQueryWithOnlyLimitOnChild(t *testing.T) { "subType": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(1), - "offset": int64(0), + "limit": uint64(1), + "offset": uint64(0), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -529,7 +529,7 @@ func TestExplainQueryWithOnlyOffsetOnChild(t *testing.T) { "selectTopNode": dataMap{ "limitNode": dataMap{ "limit": nil, - "offset": int64(2), + "offset": uint64(2), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -688,8 +688,8 @@ func TestExplainQueryWithBothLimitAndOffsetOnChild(t *testing.T) { "subType": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(2), - "offset": int64(2), + "limit": uint64(2), + "offset": uint64(2), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -827,8 +827,8 @@ func TestExplainQueryWithLimitOnChildAndBothLimitAndOffsetOnParent(t *testing.T) "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(3), - "offset": int64(1), + "limit": uint64(3), + "offset": uint64(1), "selectNode": dataMap{ "filter": nil, "typeIndexJoin": dataMap{ @@ -851,8 +851,8 @@ func TestExplainQueryWithLimitOnChildAndBothLimitAndOffsetOnParent(t *testing.T) "subType": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -1021,8 +1021,8 @@ func TestExplainQueryWithMultipleConflictingInnerLimits(t *testing.T) { "subType": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{ @@ -1201,8 +1201,8 @@ func TestExplainQueryWithMultipleConflictingInnerLimitsAndOuterLimit(t *testing. "explain": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(3), - "offset": int64(1), + "limit": uint64(3), + "offset": uint64(1), "countNode": dataMap{ "sources": []dataMap{ { @@ -1234,8 +1234,8 @@ func TestExplainQueryWithMultipleConflictingInnerLimitsAndOuterLimit(t *testing. "subType": dataMap{ "selectTopNode": dataMap{ "limitNode": dataMap{ - "limit": int64(2), - "offset": int64(0), + "limit": uint64(2), + "offset": uint64(0), "selectNode": dataMap{ "filter": nil, "scanNode": dataMap{